From c63fd695b38a34836dde588abd0321708cab5475 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 26 Sep 2018 14:14:05 +0200 Subject: [PATCH 01/20] [RCI] Add NoOpEngine for closed indices (#33903) This commit adds a new NoOpEngine implementation based on the current ReadOnlyEngine. This new implementation uses an empty DirectoryReader with no segments readers and will always returns 0 docs. The NoOpEngine is the default Engine created for IndexShards of closed indices. It expects an empty translog when it is instantiated. Relates to #33888 --- .../index/engine/NoOpEngine.java | 149 ++++++++++++ .../elasticsearch/indices/IndicesService.java | 7 + .../index/engine/NoOpEngineTests.java | 221 ++++++++++++++++++ .../index/shard/IndexShardIT.java | 38 ++- .../indices/IndicesServiceTests.java | 2 +- .../index/engine/EngineTestCase.java | 8 + 6 files changed, 423 insertions(+), 2 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java create mode 100644 server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java new file mode 100644 index 0000000000000..8e857d2606d9b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -0,0 +1,149 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.store.Directory; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.index.translog.TranslogCorruptedException; +import org.elasticsearch.index.translog.TranslogDeletionPolicy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.LongSupplier; +import java.util.stream.Stream; + +/** + * NoOpEngine is an engine implementation that does nothing but the bare minimum + * required in order to have an engine. All attempts to do something (search, + * index, get), throw {@link UnsupportedOperationException}. This does maintain + * a translog with a deletion policy so that when flushing, no translog is + * retained on disk (setting a retention size and age of 0). + * + * It's also important to notice that this does list the commits of the Store's + * Directory so that the last commit's user data can be read for the historyUUID + * and last committed segment info. + */ +public final class NoOpEngine extends ReadOnlyEngine { + + public NoOpEngine(EngineConfig engineConfig) { + super(engineConfig, null, null, true, directoryReader -> directoryReader); + boolean success = false; + try { + // The deletion policy for the translog should not keep any translogs around, so the min age/size is set to -1 + final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(-1, -1); + + // The translog is opened and closed to validate that the translog UUID from lucene is the same as the one in the translog + try (Translog translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier())) { + final int nbOperations = translog.totalOperations(); + if (nbOperations != 0) { + throw new IllegalArgumentException("Expected 0 translog operations but there were " + nbOperations); + } + } + success = true; + } catch (IOException | TranslogCorruptedException e) { + throw new EngineCreationFailureException(shardId, "failed to create engine", e); + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + protected DirectoryReader open(final Directory directory) throws IOException { + final List indexCommits = DirectoryReader.listCommits(directory); + assert indexCommits.size() == 1 : "expected only one commit point"; + IndexCommit indexCommit = indexCommits.get(indexCommits.size() - 1); + return new DirectoryReader(directory, new LeafReader[0]) { + @Override + protected DirectoryReader doOpenIfChanged() throws IOException { + return null; + } + + @Override + protected DirectoryReader doOpenIfChanged(IndexCommit commit) throws IOException { + return null; + } + + @Override + protected DirectoryReader doOpenIfChanged(IndexWriter writer, boolean applyAllDeletes) throws IOException { + return null; + } + + @Override + public long getVersion() { + return 0; + } + + @Override + public boolean isCurrent() throws IOException { + return true; + } + + @Override + public IndexCommit getIndexCommit() throws IOException { + return indexCommit; + } + + @Override + protected void doClose() throws IOException { + } + + @Override + public CacheHelper getReaderCacheHelper() { + return null; + } + }; + } + + private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy, + LongSupplier globalCheckpointSupplier) throws IOException { + final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); + final String translogUUID = loadTranslogUUIDFromLastCommit(); + // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! + return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier, + engineConfig.getPrimaryTermSupplier()); + } + + /** + * Reads the current stored translog ID from the last commit data. + */ + @Nullable + private String loadTranslogUUIDFromLastCommit() { + final Map commitUserData = getLastCommittedSegmentInfos().getUserData(); + if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) { + throw new IllegalStateException("Commit doesn't contain translog generation id"); + } + return commitUserData.get(Translog.TRANSLOG_UUID_KEY); + } + + @Override + public boolean ensureTranslogSynced(Stream locations) { + throw new UnsupportedOperationException("Translog synchronization should never be needed"); + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index cca63c015f1c7..c681f4ff144e7 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -85,6 +85,7 @@ import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.engine.NoOpEngine; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; @@ -519,6 +520,12 @@ private synchronized IndexService createIndexService(final String reason, } private EngineFactory getEngineFactory(final IndexSettings idxSettings) { + final IndexMetaData indexMetaData = idxSettings.getIndexMetaData(); + if (indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE) { + // NoOpEngine takes precedence as long as the index is closed + return NoOpEngine::new; + } + final List> engineFactories = engineFactoryProviders .stream() diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java new file mode 100644 index 0000000000000..ee76e44e97593 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java @@ -0,0 +1,221 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.engine; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.store.LockObtainFailedException; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.DocsStats; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.TranslogCorruptedException; +import org.elasticsearch.index.translog.TranslogDeletionPolicy; +import org.elasticsearch.test.IndexSettingsModule; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicLong; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class NoOpEngineTests extends EngineTestCase { + private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY); + + public void testNoopEngine() throws IOException { + engine.close(); + final NoOpEngine engine = new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, primaryTranslogDir)); + expectThrows(UnsupportedOperationException.class, () -> engine.syncFlush(null, null)); + expectThrows(UnsupportedOperationException.class, () -> engine.ensureTranslogSynced(null)); + assertThat(engine.refreshNeeded(), equalTo(false)); + assertThat(engine.shouldPeriodicallyFlush(), equalTo(false)); + engine.close(); + } + + public void testTwoNoopEngines() throws IOException { + engine.close(); + // Ensure that we can't open two noop engines for the same store + final EngineConfig engineConfig = noOpConfig(INDEX_SETTINGS, store, primaryTranslogDir); + try (NoOpEngine ignored = new NoOpEngine(engineConfig)) { + UncheckedIOException e = expectThrows(UncheckedIOException.class, () -> new NoOpEngine(engineConfig)); + assertThat(e.getCause(), instanceOf(LockObtainFailedException.class)); + } + } + + public void testNoopAfterRegularEngine() throws IOException { + int docs = randomIntBetween(1, 10); + ReplicationTracker tracker = (ReplicationTracker) engine.config().getGlobalCheckpointSupplier(); + ShardRouting routing = TestShardRouting.newShardRouting("test", shardId.id(), "node", + null, true, ShardRoutingState.STARTED, allocationId); + IndexShardRoutingTable table = new IndexShardRoutingTable.Builder(shardId).addShard(routing).build(); + tracker.updateFromMaster(1L, Collections.singleton(allocationId.getId()), table, Collections.emptySet()); + tracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + for (int i = 0; i < docs; i++) { + ParsedDocument doc = testParsedDocument("" + i, null, testDocumentWithTextField(), B_1, null); + engine.index(indexForDoc(doc)); + tracker.updateLocalCheckpoint(allocationId.getId(), i); + } + + flushAndTrimTranslog(engine); + + long localCheckpoint = engine.getLocalCheckpoint(); + long maxSeqNo = engine.getSeqNoStats(100L).getMaxSeqNo(); + engine.close(); + + final NoOpEngine noOpEngine = new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, primaryTranslogDir, tracker)); + assertThat(noOpEngine.getLocalCheckpoint(), equalTo(localCheckpoint)); + assertThat(noOpEngine.getSeqNoStats(100L).getMaxSeqNo(), equalTo(maxSeqNo)); + try (Engine.IndexCommitRef ref = noOpEngine.acquireLastIndexCommit(false)) { + try (IndexReader reader = DirectoryReader.open(ref.getIndexCommit())) { + assertThat(reader.numDocs(), equalTo(docs)); + } + } + noOpEngine.close(); + } + + public void testNoopEngineWithInvalidTranslogUUID() throws IOException { + IOUtils.close(engine, store); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore()) { + EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); + int numDocs = scaledRandomIntBetween(10, 100); + try (InternalEngine engine = createEngine(config)) { + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); + engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, + System.nanoTime(), -1, false)); + if (rarely()) { + engine.flush(); + } + globalCheckpoint.set(engine.getLocalCheckpoint()); + } + flushAndTrimTranslog(engine); + } + + final Path newTranslogDir = createTempDir(); + // A new translog will have a different UUID than the existing store/noOp engine does + Translog newTranslog = createTranslog(newTranslogDir, () -> 1L); + newTranslog.close(); + + EngineCreationFailureException e = expectThrows(EngineCreationFailureException.class, + () -> new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, newTranslogDir))); + assertThat(e.getCause(), instanceOf(TranslogCorruptedException.class)); + } + } + + public void testNoopEngineWithNonZeroTranslogOperations() throws IOException { + IOUtils.close(engine, store); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore()) { + final MergePolicy mergePolicy = NoMergePolicy.INSTANCE; + EngineConfig config = config(defaultSettings, store, createTempDir(), mergePolicy, null, null, globalCheckpoint::get); + int numDocs = scaledRandomIntBetween(10, 100); + try (InternalEngine engine = createEngine(config)) { + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); + engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, + System.nanoTime(), -1, false)); + if (rarely()) { + engine.flush(); + } + globalCheckpoint.set(engine.getLocalCheckpoint()); + } + engine.syncTranslog(); + engine.flushAndClose(); + engine.close(); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new NoOpEngine(engine.engineConfig)); + assertThat(e.getMessage(), is("Expected 0 translog operations but there were " + numDocs)); + } + } + } + + public void testNoOpEngineDocStats() throws Exception { + IOUtils.close(engine, store); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore()) { + EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); + final int numDocs = scaledRandomIntBetween(10, 3000); + int deletions = 0; + try (InternalEngine engine = createEngine(config)) { + for (int i = 0; i < numDocs; i++) { + engine.index(indexForDoc(createParsedDoc(Integer.toString(i), null))); + if (rarely()) { + engine.flush(); + } + globalCheckpoint.set(engine.getLocalCheckpoint()); + } + + for (int i = 0; i < numDocs; i++) { + if (randomBoolean()) { + String delId = Integer.toString(i); + Engine.DeleteResult result = engine.delete(new Engine.Delete("test", delId, newUid(delId), primaryTerm.get())); + assertTrue(result.isFound()); + globalCheckpoint.set(engine.getLocalCheckpoint()); + deletions += 1; + } + } + engine.waitForOpsToComplete(numDocs + deletions - 1); + flushAndTrimTranslog(engine); + engine.close(); + } + + final DocsStats expectedDocStats; + try (InternalEngine engine = createEngine(config)) { + expectedDocStats = engine.docStats(); + } + + try (NoOpEngine noOpEngine = new NoOpEngine(config)) { + assertEquals(expectedDocStats.getCount(), noOpEngine.docStats().getCount()); + assertEquals(expectedDocStats.getDeleted(), noOpEngine.docStats().getDeleted()); + assertEquals(expectedDocStats.getTotalSizeInBytes(), noOpEngine.docStats().getTotalSizeInBytes()); + assertEquals(expectedDocStats.getAverageSizeInBytes(), noOpEngine.docStats().getAverageSizeInBytes()); + } catch (AssertionError e) { + logger.error(config.getMergePolicy()); + throw e; + } + } + } + + private void flushAndTrimTranslog(final InternalEngine engine) { + engine.flush(true, true); + final TranslogDeletionPolicy deletionPolicy = engine.getTranslog().getDeletionPolicy(); + deletionPolicy.setRetentionSizeInBytes(-1); + deletionPolicy.setRetentionAgeInMillis(-1); + deletionPolicy.setMinTranslogGenerationForRecovery(engine.getTranslog().getGeneration().translogFileGeneration); + engine.flush(true, true); + } +} 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 36560dd96c627..d5f4b01aee0d4 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -60,6 +60,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.NoOpEngine; import org.elasticsearch.index.engine.SegmentsStats; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.mapper.SourceToParse; @@ -106,6 +107,7 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.NONE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; @@ -668,7 +670,7 @@ public static final IndexShard newIndexShard( } private static ShardRouting getInitializingShardRouting(ShardRouting existingShardRouting) { - ShardRouting shardRouting = TestShardRouting.newShardRouting(existingShardRouting.shardId(), + ShardRouting shardRouting = newShardRouting(existingShardRouting.shardId(), existingShardRouting.currentNodeId(), null, existingShardRouting.primary(), ShardRoutingState.INITIALIZING, existingShardRouting.allocationId()); shardRouting = shardRouting.updateUnassigned(new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, "fake recovery"), @@ -905,4 +907,38 @@ public void testShardChangesWithDefaultDocType() throws Exception { assertThat(opsFromLucene, equalTo(opsFromTranslog)); } } + + /** + * Test that the {@link org.elasticsearch.index.engine.NoOpEngine} takes precedence over other + * engine factories if the index is closed. + */ + public void testNoOpEngineFactoryTakesPrecedence() throws IOException { + final String indexName = "closed-index"; + createIndex(indexName, Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0).build()); + ensureGreen(); + + client().admin().indices().prepareClose(indexName).get(); + + final ClusterService clusterService = getInstanceFromNode(ClusterService.class); + final ClusterState clusterState = clusterService.state(); + + IndexMetaData indexMetaData = clusterState.metaData().index(indexName); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + + final ShardId shardId = new ShardId(indexMetaData.getIndex(), 0); + final DiscoveryNode node = clusterService.localNode(); + final ShardRouting routing = + newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, RecoverySource.EmptyStoreRecoverySource.INSTANCE); + + final IndexService indexService = indicesService.createIndex(indexMetaData, Collections.emptyList()); + try { + final IndexShard indexShard = indexService.createShard(routing, id -> { + }); + indexShard.markAsRecovering("store", new RecoveryState(indexShard.routingEntry(), node, null)); + indexShard.recoverFromStore(); + assertThat(indexShard.getEngine(), instanceOf(NoOpEngine.class)); + } finally { + indexService.close("test terminated", true); + } + } } diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 60dbad99795f3..1aa042edacd67 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -551,7 +551,7 @@ public void testGetEngineFactory() throws IOException { } } - public void testConflictingEngineFactories() throws IOException { + public void testConflictingEngineFactories() { final String indexName = "foobar"; final Index index = new Index(indexName, UUIDs.randomBase64UUID()); final Settings settings = Settings.builder() 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 35667b0f87a1c..681357ca5fdef 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 @@ -692,6 +692,14 @@ public EngineConfig config( tombstoneDocSupplier()); } + protected EngineConfig noOpConfig(IndexSettings indexSettings, Store store, Path translogPath) { + return noOpConfig(indexSettings, store, translogPath, null); + } + + protected EngineConfig noOpConfig(IndexSettings indexSettings, Store store, Path translogPath, LongSupplier globalCheckpointSupplier) { + return config(indexSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier); + } + protected static final BytesReference B_1 = new BytesArray(new byte[]{1}); protected static final BytesReference B_2 = new BytesArray(new byte[]{2}); protected static final BytesReference B_3 = new BytesArray(new byte[]{3}); From 54d110bf2182c54a02655f27e88cb69d79220fc7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 8 Nov 2018 11:51:03 +0100 Subject: [PATCH 02/20] [RCI] Adapt NoOpEngine to latest FrozenEngine changes Changes were made in #34357 and #36467 --- .../java/org/elasticsearch/index/engine/NoOpEngine.java | 3 ++- .../org/elasticsearch/index/engine/NoOpEngineTests.java | 8 ++++---- .../java/org/elasticsearch/index/shard/IndexShardIT.java | 1 - 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index 8e857d2606d9b..265aac34ea079 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -75,7 +75,8 @@ public NoOpEngine(EngineConfig engineConfig) { } @Override - protected DirectoryReader open(final Directory directory) throws IOException { + protected DirectoryReader open(final IndexCommit commit) throws IOException { + final Directory directory = commit.getDirectory(); final List indexCommits = DirectoryReader.listCommits(directory); assert indexCommits.size() == 1 : "expected only one commit point"; IndexCommit indexCommit = indexCommits.get(indexCommits.size() - 1); diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java index ee76e44e97593..bac67b10dfb36 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java @@ -116,7 +116,7 @@ public void testNoopEngineWithInvalidTranslogUUID() throws IOException { for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (rarely()) { engine.flush(); } @@ -147,7 +147,7 @@ public void testNoopEngineWithNonZeroTranslogOperations() throws IOException { for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false)); + System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); if (rarely()) { engine.flush(); } @@ -167,7 +167,8 @@ public void testNoOpEngineDocStats() throws Exception { IOUtils.close(engine, store); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); try (Store store = createStore()) { - EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); + Path translogPath = createTempDir(); + EngineConfig config = config(defaultSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); final int numDocs = scaledRandomIntBetween(10, 3000); int deletions = 0; try (InternalEngine engine = createEngine(config)) { @@ -190,7 +191,6 @@ public void testNoOpEngineDocStats() throws Exception { } engine.waitForOpsToComplete(numDocs + deletions - 1); flushAndTrimTranslog(engine); - engine.close(); } final DocsStats expectedDocStats; 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 d5f4b01aee0d4..80ce592e47d40 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -38,7 +38,6 @@ import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedRunnable; From cae41551a2e975cc9c7bc470808b384b27c0dc6b Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 23 Jan 2019 09:29:13 +0100 Subject: [PATCH 03/20] Relax NoOpEngine constraints (#37413) When a NoOpEngine is instanciated, the current implementation verifies that the translog contains no operations and that it contains the same UUID as the last Lucene commit data.We can relax those two constraints because the Close Index API now ensure that all translog operations are flushed before closing a shard. The detection of coherence between translog UUID / Lucene commit data is not specific to NoOpEngine, and is already done by IndexShard.innerOpenEngineAndTranslog(). Related to #33888 --- .../index/engine/NoOpEngine.java | 68 ++----------------- .../index/engine/NoOpEngineRecoveryTests.java | 54 +++++++++++++++ .../index/engine/NoOpEngineTests.java | 63 ----------------- 3 files changed, 58 insertions(+), 127 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index 265aac34ea079..8eec141975312 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -24,54 +24,20 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; import org.apache.lucene.store.Directory; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogConfig; -import org.elasticsearch.index.translog.TranslogCorruptedException; -import org.elasticsearch.index.translog.TranslogDeletionPolicy; import java.io.IOException; import java.util.List; -import java.util.Map; -import java.util.function.LongSupplier; -import java.util.stream.Stream; +import java.util.function.Function; /** * NoOpEngine is an engine implementation that does nothing but the bare minimum * required in order to have an engine. All attempts to do something (search, - * index, get), throw {@link UnsupportedOperationException}. This does maintain - * a translog with a deletion policy so that when flushing, no translog is - * retained on disk (setting a retention size and age of 0). - * - * It's also important to notice that this does list the commits of the Store's - * Directory so that the last commit's user data can be read for the historyUUID - * and last committed segment info. + * index, get), throw {@link UnsupportedOperationException}. */ public final class NoOpEngine extends ReadOnlyEngine { - public NoOpEngine(EngineConfig engineConfig) { - super(engineConfig, null, null, true, directoryReader -> directoryReader); - boolean success = false; - try { - // The deletion policy for the translog should not keep any translogs around, so the min age/size is set to -1 - final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(-1, -1); - - // The translog is opened and closed to validate that the translog UUID from lucene is the same as the one in the translog - try (Translog translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier())) { - final int nbOperations = translog.totalOperations(); - if (nbOperations != 0) { - throw new IllegalArgumentException("Expected 0 translog operations but there were " + nbOperations); - } - } - success = true; - } catch (IOException | TranslogCorruptedException e) { - throw new EngineCreationFailureException(shardId, "failed to create engine", e); - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(this); - } - } + public NoOpEngine(EngineConfig config) { + super(config, null, null, true, Function.identity()); } @Override @@ -121,30 +87,4 @@ public CacheHelper getReaderCacheHelper() { } }; } - - private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy, - LongSupplier globalCheckpointSupplier) throws IOException { - final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); - final String translogUUID = loadTranslogUUIDFromLastCommit(); - // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! - return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier, - engineConfig.getPrimaryTermSupplier()); - } - - /** - * Reads the current stored translog ID from the last commit data. - */ - @Nullable - private String loadTranslogUUIDFromLastCommit() { - final Map commitUserData = getLastCommittedSegmentInfos().getUserData(); - if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) { - throw new IllegalStateException("Commit doesn't contain translog generation id"); - } - return commitUserData.get(Translog.TRANSLOG_UUID_KEY); - } - - @Override - public boolean ensureTranslogSynced(Stream locations) { - throw new UnsupportedOperationException("Translog synchronization should never be needed"); - } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java new file mode 100644 index 0000000000000..7e8f18dd005fc --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java @@ -0,0 +1,54 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.index.engine; + +import org.elasticsearch.cluster.routing.RecoverySource.ExistingStoreRecoverySource; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardTestCase; + +import java.io.IOException; + +import static org.elasticsearch.cluster.routing.ShardRoutingHelper.initWithSameId; + +public class NoOpEngineRecoveryTests extends IndexShardTestCase { + + public void testRecoverFromNoOp() throws IOException { + final int nbDocs = scaledRandomIntBetween(1, 100); + + final IndexShard indexShard = newStartedShard(true); + for (int i = 0; i < nbDocs; i++) { + indexDoc(indexShard, "_doc", String.valueOf(i)); + } + indexShard.close("test", true); + + final ShardRouting shardRouting = indexShard.routingEntry(); + IndexShard primary = reinitShard(indexShard, initWithSameId(shardRouting, ExistingStoreRecoverySource.INSTANCE), NoOpEngine::new); + recoverShardFromStore(primary); + assertEquals(primary.seqNoStats().getMaxSeqNo(), primary.getMaxSeqNoOfUpdatesOrDeletes()); + assertEquals(nbDocs, primary.docStats().getCount()); + + IndexShard replica = newShard(false, Settings.EMPTY, NoOpEngine::new); + recoverReplica(replica, primary, true); + assertEquals(replica.seqNoStats().getMaxSeqNo(), replica.getMaxSeqNoOfUpdatesOrDeletes()); + assertEquals(nbDocs, replica.docStats().getCount()); + closeShards(primary, replica); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java index bac67b10dfb36..9a088a85ac7a1 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java @@ -21,14 +21,12 @@ import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.store.LockObtainFailedException; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; @@ -37,8 +35,6 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogCorruptedException; import org.elasticsearch.index.translog.TranslogDeletionPolicy; import org.elasticsearch.test.IndexSettingsModule; @@ -50,7 +46,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; public class NoOpEngineTests extends EngineTestCase { private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY); @@ -59,7 +54,6 @@ public void testNoopEngine() throws IOException { engine.close(); final NoOpEngine engine = new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, primaryTranslogDir)); expectThrows(UnsupportedOperationException.class, () -> engine.syncFlush(null, null)); - expectThrows(UnsupportedOperationException.class, () -> engine.ensureTranslogSynced(null)); assertThat(engine.refreshNeeded(), equalTo(false)); assertThat(engine.shouldPeriodicallyFlush(), equalTo(false)); engine.close(); @@ -106,63 +100,6 @@ public void testNoopAfterRegularEngine() throws IOException { noOpEngine.close(); } - public void testNoopEngineWithInvalidTranslogUUID() throws IOException { - IOUtils.close(engine, store); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - try (Store store = createStore()) { - EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); - int numDocs = scaledRandomIntBetween(10, 100); - try (InternalEngine engine = createEngine(config)) { - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); - if (rarely()) { - engine.flush(); - } - globalCheckpoint.set(engine.getLocalCheckpoint()); - } - flushAndTrimTranslog(engine); - } - - final Path newTranslogDir = createTempDir(); - // A new translog will have a different UUID than the existing store/noOp engine does - Translog newTranslog = createTranslog(newTranslogDir, () -> 1L); - newTranslog.close(); - - EngineCreationFailureException e = expectThrows(EngineCreationFailureException.class, - () -> new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, newTranslogDir))); - assertThat(e.getCause(), instanceOf(TranslogCorruptedException.class)); - } - } - - public void testNoopEngineWithNonZeroTranslogOperations() throws IOException { - IOUtils.close(engine, store); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - try (Store store = createStore()) { - final MergePolicy mergePolicy = NoMergePolicy.INSTANCE; - EngineConfig config = config(defaultSettings, store, createTempDir(), mergePolicy, null, null, globalCheckpoint::get); - int numDocs = scaledRandomIntBetween(10, 100); - try (InternalEngine engine = createEngine(config)) { - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA, - System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0)); - if (rarely()) { - engine.flush(); - } - globalCheckpoint.set(engine.getLocalCheckpoint()); - } - engine.syncTranslog(); - engine.flushAndClose(); - engine.close(); - - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new NoOpEngine(engine.engineConfig)); - assertThat(e.getMessage(), is("Expected 0 translog operations but there were " + numDocs)); - } - } - } - public void testNoOpEngineDocStats() throws Exception { IOUtils.close(engine, store); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); From e53a9beec1dda08093bb074f16c4fd425e9f3f5f Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 29 Jan 2019 15:36:53 +0100 Subject: [PATCH 04/20] Fix compilation error in IndexShardIT after merge with master --- .../test/java/org/elasticsearch/index/shard/IndexShardIT.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 80ce592e47d40..5565567c2197a 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -931,8 +931,7 @@ public void testNoOpEngineFactoryTakesPrecedence() throws IOException { final IndexService indexService = indicesService.createIndex(indexMetaData, Collections.emptyList()); try { - final IndexShard indexShard = indexService.createShard(routing, id -> { - }); + final IndexShard indexShard = indexService.createShard(routing, id -> {}, (s, leases, listener) -> {}); indexShard.markAsRecovering("store", new RecoveryState(indexShard.routingEntry(), node, null)); indexShard.recoverFromStore(); assertThat(indexShard.getEngine(), instanceOf(NoOpEngine.class)); From 02cc73017081d29d86a42d1c81b5a26720a91756 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 4 Feb 2019 14:19:52 +0100 Subject: [PATCH 05/20] Allow shards of closed indices to be replicated as regular shards (#38024) This commit allows shards of indices in CLOSE state to be replicated as normal shards. It changes the MetaDataIndexStateService so that index routing tables of closed indices are kept in cluster state when the index is closed. Index routing tables are modified so that shard routings are reinitialized with the INDEX_CLOSED unassigned information. The IndicesClusterStateService is modified to remove IndexService instances of closed or reopened indices. In combination with the ShardRouting being in INITIALIZING state the shards are recreated on the data nodes to reflect the new state. If the index state is closed, the IndexShard instances will be created using the NoOpEngine as the engine implementation. This commit also mutes two tests that rely on the fact that shard locks are released when an index is closed, which is not the case anymore with replicated closed indices (actually the locks are released but reacquired once the shard is reinitialized after being closed). These tests will be adapted in follow up PRs. Finally, many things will require to be adapted or improved in follow up PRs (see #33888) but this is the first big step towards replicated closed indices. Relates to #33888 --- .../metadata/MetaDataIndexStateService.java | 6 ++- .../cluster/routing/IndexRoutingTable.java | 7 +++ .../cluster/routing/RoutingTable.java | 7 +++ .../cluster/routing/UnassignedInfo.java | 8 +++- .../index/engine/NoOpEngine.java | 3 +- .../cluster/IndicesClusterStateService.java | 47 ++++++++++++------- .../elasticsearch/search/SearchService.java | 2 +- .../MetaDataIndexStateServiceTests.java | 41 +++++++++++----- .../cluster/routing/UnassignedInfoTests.java | 20 +++++++- .../gateway/GatewayIndexStateIT.java | 16 ++++--- .../index/shard/IndexShardIT.java | 20 +++----- .../indices/IndicesServiceTests.java | 2 + .../indices/state/CloseIndexIT.java | 3 +- .../indices/state/SimpleIndexStateIT.java | 4 +- .../search/scroll/SearchScrollIT.java | 11 ++--- .../SharedClusterSnapshotRestoreIT.java | 4 +- 16 files changed, 132 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index 0781cab1fe757..3e9143320c53c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -410,14 +410,16 @@ static ClusterState closeRoutingTable(final ClusterState currentState, } logger.debug("closing index {} succeeded", index); - blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID).addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); - routingTable.remove(index.getName()); + blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID); + blocks.addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + routingTable.addAsFromOpenToClose(metadata.getSafe(index)); closedIndices.add(index.getName()); } catch (final IndexNotFoundException e) { logger.debug("index {} has been deleted since it was blocked before closing, ignoring", index); } } + logger.info("completed closing of indices {}", closedIndices); return ClusterState.builder(currentState).blocks(blocks).metaData(metadata).routingTable(routingTable.build()).build(); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index cf1235c8f2158..195ae2cce25b8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -358,6 +358,13 @@ public Builder initializeAsFromCloseToOpen(IndexMetaData indexMetaData) { return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_REOPENED, null)); } + /** + * Initializes a new empty index, as as a result of closing an opened index. + */ + public Builder initializeAsFromOpenToClose(IndexMetaData indexMetaData) { + return initializeEmpty(indexMetaData, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CLOSED, null)); + } + /** * Initializes a new empty index, to be restored from a snapshot */ diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 0d5ee132ffa9b..0c6080029895b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -540,6 +540,13 @@ public Builder addAsFromCloseToOpen(IndexMetaData indexMetaData) { return this; } + public Builder addAsFromOpenToClose(IndexMetaData indexMetaData) { + assert indexMetaData.getState() == IndexMetaData.State.CLOSE; + IndexRoutingTable.Builder indexRoutingBuilder = new IndexRoutingTable.Builder(indexMetaData.getIndex()) + .initializeAsFromOpenToClose(indexMetaData); + return add(indexRoutingBuilder); + } + public Builder addAsRestore(IndexMetaData indexMetaData, SnapshotRecoverySource recoverySource) { IndexRoutingTable.Builder indexRoutingBuilder = new IndexRoutingTable.Builder(indexMetaData.getIndex()) .initializeAsRestore(indexMetaData, recoverySource); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java index f8afbeb449361..ec7b053764afc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/UnassignedInfo.java @@ -118,7 +118,11 @@ public enum Reason { /** * Forced manually to allocate */ - MANUAL_ALLOCATION + MANUAL_ALLOCATION, + /** + * Unassigned as a result of closing an index. + */ + INDEX_CLOSED } /** @@ -269,6 +273,8 @@ public UnassignedInfo(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().before(Version.V_6_0_0_beta2) && reason == Reason.MANUAL_ALLOCATION) { out.writeByte((byte) Reason.ALLOCATION_FAILED.ordinal()); + } else if (out.getVersion().before(Version.V_7_0_0) && reason == Reason.INDEX_CLOSED) { + out.writeByte((byte) Reason.REINITIALIZED.ordinal()); } else { out.writeByte((byte) reason.ordinal()); } diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index 8eec141975312..fe1ad7a1a144f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -44,8 +44,7 @@ public NoOpEngine(EngineConfig config) { protected DirectoryReader open(final IndexCommit commit) throws IOException { final Directory directory = commit.getDirectory(); final List indexCommits = DirectoryReader.listCommits(directory); - assert indexCommits.size() == 1 : "expected only one commit point"; - IndexCommit indexCommit = indexCommits.get(indexCommits.size() - 1); + final IndexCommit indexCommit = indexCommits.get(indexCommits.size() - 1); return new DirectoryReader(directory, new LeafReader[0]) { @Override protected DirectoryReader doOpenIfChanged() throws IOException { diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 57ec87d1c6493..c0fbd1272b973 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -98,6 +98,7 @@ import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.FAILURE; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.NO_LONGER_ASSIGNED; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.REOPENED; public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateApplier { private static final Logger logger = LogManager.getLogger(IndicesClusterStateService.class); @@ -240,7 +241,7 @@ public synchronized void applyClusterState(final ClusterChangedEvent event) { deleteIndices(event); // also deletes shards of deleted indices - removeUnallocatedIndices(event); // also removes shards of removed indices + removeIndices(event); // also removes shards of removed indices failMissingShards(state); @@ -352,17 +353,18 @@ protected void doRun() throws Exception { } /** - * Removes indices that have no shards allocated to this node. This does not delete the shard data as we wait for enough - * shard copies to exist in the cluster before deleting shard data (triggered by {@link org.elasticsearch.indices.store.IndicesStore}). + * Removes indices that have no shards allocated to this node or indices whose state has changed. This does not delete the shard data + * as we wait for enough shard copies to exist in the cluster before deleting shard data (triggered by + * {@link org.elasticsearch.indices.store.IndicesStore}). * * @param event the cluster changed event */ - private void removeUnallocatedIndices(final ClusterChangedEvent event) { + private void removeIndices(final ClusterChangedEvent event) { final ClusterState state = event.state(); final String localNodeId = state.nodes().getLocalNodeId(); assert localNodeId != null; - Set indicesWithShards = new HashSet<>(); + final Set indicesWithShards = new HashSet<>(); RoutingNode localRoutingNode = state.getRoutingNodes().node(localNodeId); if (localRoutingNode != null) { // null e.g. if we are not a data node for (ShardRouting shardRouting : localRoutingNode) { @@ -371,20 +373,27 @@ private void removeUnallocatedIndices(final ClusterChangedEvent event) { } for (AllocatedIndex indexService : indicesService) { - Index index = indexService.index(); - if (indicesWithShards.contains(index) == false) { + final Index index = indexService.index(); + final IndexMetaData indexMetaData = state.metaData().index(index); + final IndexMetaData existingMetaData = indexService.getIndexSettings().getIndexMetaData(); + + AllocatedIndices.IndexRemovalReason reason = null; + if (indexMetaData != null && indexMetaData.getState() != existingMetaData.getState()) { + reason = indexMetaData.getState() == IndexMetaData.State.CLOSE ? CLOSED : REOPENED; + } else if (indicesWithShards.contains(index) == false) { // if the cluster change indicates a brand new cluster, we only want // to remove the in-memory structures for the index and not delete the // contents on disk because the index will later be re-imported as a // dangling index - final IndexMetaData indexMetaData = state.metaData().index(index); assert indexMetaData != null || event.isNewCluster() : "index " + index + " does not exist in the cluster state, it should either " + "have been deleted or the cluster must be new"; - final AllocatedIndices.IndexRemovalReason reason = - indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE ? CLOSED : NO_LONGER_ASSIGNED; - logger.debug("{} removing index, [{}]", index, reason); - indicesService.removeIndex(index, reason, "removing index (no shards allocated)"); + reason = indexMetaData != null && indexMetaData.getState() == IndexMetaData.State.CLOSE ? CLOSED : NO_LONGER_ASSIGNED; + } + + if (reason != null) { + logger.debug("{} removing index ({})", index, reason); + indicesService.removeIndex(index, reason, "removing index (" + reason + ")"); } } } @@ -595,7 +604,7 @@ private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard ClusterState clusterState) { final ShardRouting currentRoutingEntry = shard.routingEntry(); assert currentRoutingEntry.isSameAllocation(shardRouting) : - "local shard has a different allocation id but wasn't cleaning by removeShards. " + "local shard has a different allocation id but wasn't cleaned by removeShards. " + "cluster state: " + shardRouting + " local: " + currentRoutingEntry; final long primaryTerm; @@ -730,7 +739,7 @@ private void failAndRemoveShard(ShardRouting shardRouting, boolean sendShardFail private void sendFailShard(ShardRouting shardRouting, String message, @Nullable Exception failure, ClusterState state) { try { logger.warn(() -> new ParameterizedMessage( - "[{}] marking and sending shard failed due to [{}]", shardRouting.shardId(), message), failure); + "{} marking and sending shard failed due to [{}]", shardRouting.shardId(), message), failure); failedShardsCache.put(shardRouting.shardId(), shardRouting); shardStateAction.localShardFailed(shardRouting, message, failure, SHARD_STATE_ACTION_LISTENER, state); } catch (Exception inner) { @@ -931,7 +940,7 @@ enum IndexRemovalReason { DELETED, /** - * The index have been closed. The index should be removed and all associated resources released. Persistent parts of the index + * The index has been closed. The index should be removed and all associated resources released. Persistent parts of the index * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. */ CLOSED, @@ -941,7 +950,13 @@ enum IndexRemovalReason { * Persistent parts of the index like the shards files, state and transaction logs are kept around in the * case of a disaster recovery. */ - FAILURE + FAILURE, + + /** + * The index has been reopened. The index should be removed and all associated resources released. Persistent parts of the index + * like the shards files, state and transaction logs are kept around in the case of a disaster recovery. + */ + REOPENED, } } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index a14b4a328775c..b8bde4e4f5213 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -268,7 +268,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem // it's fine to keep the contexts open if the index is still "alive" // unfortunately we don't have a clear way to signal today why an index is closed. // to release memory and let references to the filesystem go etc. - if (reason == IndexRemovalReason.DELETED || reason == IndexRemovalReason.CLOSED) { + if (reason == IndexRemovalReason.DELETED || reason == IndexRemovalReason.CLOSED || reason == IndexRemovalReason.REOPENED) { freeAllContextForIndex(index); } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index 56ee25ee5febb..4108c542d0cc5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.shards.ClusterShardLimitIT; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ValidationException; @@ -210,7 +211,14 @@ public void testAddIndexClosedBlocks() { for (Index index : indices) { assertTrue(blockedIndices.containsKey(index)); if (mixedVersions) { - assertIsClosed(index.getName(), updatedState); + assertThat(updatedState.metaData().index(index).getState(), is(IndexMetaData.State.CLOSE)); + assertTrue(updatedState.blocks().hasIndexBlock(index.getName(), MetaDataIndexStateService.INDEX_CLOSED_BLOCK)); + assertThat("Index " + index + " must have only 1 block with id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID, + updatedState.blocks().indices().getOrDefault(index.getName(), emptySet()).stream().filter(clusterBlock -> + clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); + + final IndexRoutingTable indexRoutingTable = updatedState.routingTable().index(index); + assertThat(indexRoutingTable, nullValue()); } else { assertHasBlock(index.getName(), updatedState, blockedIndices.get(index)); } @@ -346,19 +354,18 @@ private static ClusterState addIndex(final ClusterState currentState, final ClusterState.Builder clusterStateBuilder = ClusterState.builder(currentState); clusterStateBuilder.metaData(MetaData.builder(currentState.metaData()).put(indexMetaData, true)); - if (state == IndexMetaData.State.OPEN) { - final IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(indexMetaData.getIndex()); - for (int j = 0; j < indexMetaData.getNumberOfShards(); j++) { - ShardId shardId = new ShardId(indexMetaData.getIndex(), j); - IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); - indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), true, ShardRoutingState.STARTED)); - for (int k = 0; k < indexMetaData.getNumberOfReplicas(); k++) { - indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), false, ShardRoutingState.STARTED)); - } - indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build()); + final IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(indexMetaData.getIndex()); + for (int j = 0; j < indexMetaData.getNumberOfShards(); j++) { + ShardId shardId = new ShardId(indexMetaData.getIndex(), j); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), true, ShardRoutingState.STARTED)); + for (int k = 0; k < indexMetaData.getNumberOfReplicas(); k++) { + indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), false, ShardRoutingState.STARTED)); } - clusterStateBuilder.routingTable(RoutingTable.builder(currentState.routingTable()).add(indexRoutingTable).build()); + indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build()); } + clusterStateBuilder.routingTable(RoutingTable.builder(currentState.routingTable()).add(indexRoutingTable).build()); + if (block != null) { clusterStateBuilder.blocks(ClusterBlocks.builder().blocks(currentState.blocks()).addIndexBlock(index, block)); } @@ -372,11 +379,19 @@ private static void assertIsOpened(final String indexName, final ClusterState cl private static void assertIsClosed(final String indexName, final ClusterState clusterState) { assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.CLOSE)); - assertThat(clusterState.routingTable().index(indexName), nullValue()); assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); assertThat("Index " + indexName + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", clusterState.blocks().indices().getOrDefault(indexName, emptySet()).stream() .filter(clusterBlock -> clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); + + final IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(indexName); + assertThat(indexRoutingTable, notNullValue()); + + for(IndexShardRoutingTable shardRoutingTable : indexRoutingTable) { + assertThat(shardRoutingTable.shards().stream().allMatch(ShardRouting::unassigned), is(true)); + assertThat(shardRoutingTable.shards().stream().map(ShardRouting::unassignedInfo).map(UnassignedInfo::getReason) + .allMatch(info -> info == UnassignedInfo.Reason.INDEX_CLOSED), is(true)); + } } private static void assertHasBlock(final String indexName, final ClusterState clusterState, final ClusterBlock closingBlock) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 16cde1a990907..bc3191c14dfba 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.test.VersionUtils; import java.io.IOException; import java.nio.ByteBuffer; @@ -54,6 +55,7 @@ import static org.hamcrest.Matchers.nullValue; public class UnassignedInfoTests extends ESAllocationTestCase { + public void testReasonOrdinalOrder() { UnassignedInfo.Reason[] order = new UnassignedInfo.Reason[]{ UnassignedInfo.Reason.INDEX_CREATED, @@ -70,7 +72,8 @@ public void testReasonOrdinalOrder() { UnassignedInfo.Reason.REALLOCATED_REPLICA, UnassignedInfo.Reason.PRIMARY_FAILED, UnassignedInfo.Reason.FORCED_EMPTY_PRIMARY, - UnassignedInfo.Reason.MANUAL_ALLOCATION,}; + UnassignedInfo.Reason.MANUAL_ALLOCATION, + UnassignedInfo.Reason.INDEX_CLOSED,}; for (int i = 0; i < order.length; i++) { assertThat(order[i].ordinal(), equalTo(i)); } @@ -95,6 +98,21 @@ public void testSerialization() throws Exception { assertThat(read.getNumFailedAllocations(), equalTo(meta.getNumFailedAllocations())); } + public void testBwcSerialization() throws Exception { + final UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CLOSED, "message"); + BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, VersionUtils.getPreviousVersion(Version.V_7_0_0))); + unassignedInfo.writeTo(out); + out.close(); + + UnassignedInfo read = new UnassignedInfo(out.bytes().streamInput()); + assertThat(read.getReason(), equalTo(UnassignedInfo.Reason.REINITIALIZED)); + assertThat(read.getUnassignedTimeInMillis(), equalTo(unassignedInfo.getUnassignedTimeInMillis())); + assertThat(read.getMessage(), equalTo(unassignedInfo.getMessage())); + assertThat(read.getDetails(), equalTo(unassignedInfo.getDetails())); + assertThat(read.getNumFailedAllocations(), equalTo(unassignedInfo.getNumFailedAllocations())); + } + public void testIndexCreated() { MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)) diff --git a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index ebdae985a39c7..541a24247473a 100644 --- a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -54,9 +54,11 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) @@ -113,11 +115,11 @@ public void testSimpleOpenClose() throws Exception { client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get(); logger.info("--> closing test index..."); - client().admin().indices().prepareClose("test").get(); + assertAcked(client().admin().indices().prepareClose("test")); stateResponse = client().admin().cluster().prepareState().execute().actionGet(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test"), notNullValue()); logger.info("--> verifying that the state is green"); ensureGreen(); @@ -136,7 +138,7 @@ public void testSimpleOpenClose() throws Exception { ensureGreen(); logger.info("--> opening the first index again..."); - client().admin().indices().prepareOpen("test").execute().actionGet(); + assertAcked(client().admin().indices().prepareOpen("test")); logger.info("--> verifying that the state is green"); ensureGreen(); @@ -152,10 +154,10 @@ public void testSimpleOpenClose() throws Exception { assertThat(getResponse.isExists(), equalTo(true)); logger.info("--> closing test index..."); - client().admin().indices().prepareClose("test").execute().actionGet(); + assertAcked(client().admin().indices().prepareClose("test")); stateResponse = client().admin().cluster().prepareState().execute().actionGet(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test"), notNullValue()); logger.info("--> restarting nodes..."); internalCluster().fullRestart(); @@ -252,11 +254,11 @@ public void testTwoNodesSingleDoc() throws Exception { } logger.info("--> closing test index..."); - client().admin().indices().prepareClose("test").execute().actionGet(); + assertAcked(client().admin().indices().prepareClose("test")); ClusterStateResponse stateResponse = client().admin().cluster().prepareState().execute().actionGet(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test"), notNullValue()); logger.info("--> opening the index..."); client().admin().indices().prepareOpen("test").execute().actionGet(); 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 250af071cf6fe..71b5da28274aa 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -276,6 +276,8 @@ public void testExpectedShardSizeIsPresent() throws InterruptedException { assertTrue(test > 0); } + // NORELEASE This test need to be adapted for replicated closed indices + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/33888") public void testIndexCanChangeCustomDataPath() throws Exception { Environment env = getInstanceFromNode(Environment.class); Path idxPath = env.sharedDataFile().resolve(randomAlphaOfLength(10)); @@ -922,27 +924,17 @@ public void testNoOpEngineFactoryTakesPrecedence() throws IOException { createIndex(indexName, Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0).build()); ensureGreen(); - client().admin().indices().prepareClose(indexName).get(); + assertAcked(client().admin().indices().prepareClose(indexName)); final ClusterService clusterService = getInstanceFromNode(ClusterService.class); final ClusterState clusterState = clusterService.state(); - IndexMetaData indexMetaData = clusterState.metaData().index(indexName); + final IndexMetaData indexMetaData = clusterState.metaData().index(indexName); final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - - final ShardId shardId = new ShardId(indexMetaData.getIndex(), 0); - final DiscoveryNode node = clusterService.localNode(); - final ShardRouting routing = - newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, RecoverySource.EmptyStoreRecoverySource.INSTANCE); - final IndexService indexService = indicesService.createIndex(indexMetaData, Collections.emptyList()); - try { - final IndexShard indexShard = indexService.createShard(routing, id -> {}, (s, leases, listener) -> {}); - indexShard.markAsRecovering("store", new RecoveryState(indexShard.routingEntry(), node, null)); - indexShard.recoverFromStore(); + + for (IndexShard indexShard : indexService) { assertThat(indexShard.getEngine(), instanceOf(NoOpEngine.class)); - } finally { - indexService.close("test terminated", true); } } } diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 1aa042edacd67..8ee47f50a7417 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -271,6 +271,8 @@ public void testDeleteIndexStore() throws Exception { ensureGreen("test"); } + // NORELEASE This test need to be adapted for replicated closed indices + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/33888") public void testPendingTasks() throws Exception { IndicesService indicesService = getIndicesService(); IndexService test = createIndex("test"); diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java index 1d32283c6cb94..ca3f6e694097d 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java @@ -49,7 +49,6 @@ import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; public class CloseIndexIT extends ESIntegTestCase { @@ -310,7 +309,7 @@ static void assertIndexIsClosed(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); for (String index : indices) { assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.CLOSE)); - assertThat(clusterState.routingTable().index(index), nullValue()); + assertThat(clusterState.routingTable().index(index), notNullValue()); assertThat(clusterState.blocks().hasIndexBlock(index, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); assertThat("Index " + index + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", clusterState.blocks().indices().getOrDefault(index, emptySet()).stream() diff --git a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java index 1cc2d3e68e2ae..050d77a223101 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java @@ -36,7 +36,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.notNullValue; @ESIntegTestCase.ClusterScope(minNumDataNodes = 2) public class SimpleIndexStateIT extends ESIntegTestCase { @@ -65,7 +65,7 @@ public void testSimpleOpenClose() { stateResponse = client().admin().cluster().prepareState().get(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test"), notNullValue()); logger.info("--> trying to index into a closed index ..."); try { diff --git a/server/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java b/server/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java index 9fb05af2040b3..e0ae78dff3466 100644 --- a/server/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java +++ b/server/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java @@ -521,11 +521,10 @@ public void testStringSortMissingAscTerminates() throws Exception { assertThat(response.getHits().getHits().length, equalTo(0)); } - public void testCloseAndReopenOrDeleteWithActiveScroll() throws IOException { + public void testCloseAndReopenOrDeleteWithActiveScroll() { createIndex("test"); for (int i = 0; i < 100; i++) { - client().prepareIndex("test", "type1", Integer.toString(i)).setSource(jsonBuilder().startObject().field("field", i).endObject()) - .get(); + client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field", i).get(); } refresh(); SearchResponse searchResponse = client().prepareSearch() @@ -541,11 +540,11 @@ public void testCloseAndReopenOrDeleteWithActiveScroll() throws IOException { assertThat(((Number) hit.getSortValues()[0]).longValue(), equalTo(counter++)); } if (randomBoolean()) { - client().admin().indices().prepareClose("test").get(); - client().admin().indices().prepareOpen("test").get(); + assertAcked(client().admin().indices().prepareClose("test")); + assertAcked(client().admin().indices().prepareOpen("test")); ensureGreen("test"); } else { - client().admin().indices().prepareDelete("test").get(); + assertAcked(client().admin().indices().prepareDelete("test")); } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 78892516c4ade..68def650b6d4b 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -1550,7 +1550,7 @@ public void testSnapshotClosedIndex() throws Exception { assertAcked(client.admin().indices().prepareClose("test-idx-closed")); ClusterStateResponse stateResponse = client.admin().cluster().prepareState().get(); assertThat(stateResponse.getState().metaData().index("test-idx-closed").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test-idx-closed"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test-idx-closed"), notNullValue()); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap") @@ -3749,7 +3749,7 @@ public void testRestoreIncreasesPrimaryTerms() { final IndexMetaData restoredIndexMetaData = client().admin().cluster().prepareState().clear().setIndices(indexName) .setMetaData(true).get().getState().metaData().index(indexName); for (int shardId = 0; shardId < numPrimaries; shardId++) { - assertThat(restoredIndexMetaData.primaryTerm(shardId), equalTo(primaryTerms.get(shardId) + 1)); + assertThat(restoredIndexMetaData.primaryTerm(shardId), greaterThan(primaryTerms.get(shardId))); } } From b9becdddc7441de8d1ca282cfa8871cd32b2595f Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 11 Feb 2019 11:39:14 +0100 Subject: [PATCH 06/20] Adapt testPendingTasks() for replicated closed indices (#38326) Relates to #33888 --- .../indices/IndicesServiceTests.java | 78 ++++++++++--------- 1 file changed, 42 insertions(+), 36 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 8ee47f50a7417..6ec5a4c3d65d7 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -80,7 +80,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -271,60 +270,67 @@ public void testDeleteIndexStore() throws Exception { ensureGreen("test"); } - // NORELEASE This test need to be adapted for replicated closed indices - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/33888") public void testPendingTasks() throws Exception { - IndicesService indicesService = getIndicesService(); - IndexService test = createIndex("test"); + final IndexService indexService = createIndex("test"); + final Index index = indexService.index(); + final IndexSettings indexSettings = indexService.getIndexSettings(); - assertTrue(test.hasShard(0)); - ShardPath path = test.getShardOrNull(0).shardPath(); - assertTrue(test.getShardOrNull(0).routingEntry().started()); - ShardPath shardPath = ShardPath.loadShardPath(logger, getNodeEnvironment(), new ShardId(test.index(), 0), test.getIndexSettings()); - assertEquals(shardPath, path); - try { - indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); - fail("can't get lock"); - } catch (ShardLockObtainFailedException ex) { + final IndexShard indexShard = indexService.getShardOrNull(0); + assertNotNull(indexShard); + assertTrue(indexShard.routingEntry().started()); - } - assertTrue(path.exists()); + final ShardPath shardPath = indexShard.shardPath(); + assertEquals(ShardPath.loadShardPath(logger, getNodeEnvironment(), indexShard.shardId(), indexSettings), shardPath); + + final IndicesService indicesService = getIndicesService(); + expectThrows(ShardLockObtainFailedException.class, () -> + indicesService.processPendingDeletes(index, indexSettings, TimeValue.timeValueMillis(0))); + assertTrue(shardPath.exists()); int numPending = 1; if (randomBoolean()) { - indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings()); + indicesService.addPendingDelete(indexShard.shardId(), indexSettings); } else { if (randomBoolean()) { numPending++; - indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings()); + indicesService.addPendingDelete(indexShard.shardId(), indexSettings); } - indicesService.addPendingDelete(test.index(), test.getIndexSettings()); + indicesService.addPendingDelete(index, indexSettings); } + assertAcked(client().admin().indices().prepareClose("test")); - assertTrue(path.exists()); + assertTrue(shardPath.exists()); + ensureGreen("test"); - assertEquals(indicesService.numPendingDeletes(test.index()), numPending); + assertEquals(indicesService.numPendingDeletes(index), numPending); assertTrue(indicesService.hasUncompletedPendingDeletes()); - // shard lock released... we can now delete - indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); - assertEquals(indicesService.numPendingDeletes(test.index()), 0); - assertFalse(indicesService.hasUncompletedPendingDeletes()); - assertFalse(path.exists()); + expectThrows(ShardLockObtainFailedException.class, () -> + indicesService.processPendingDeletes(index, indexSettings, TimeValue.timeValueMillis(0))); - if (randomBoolean()) { - indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings()); - indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings()); - indicesService.addPendingDelete(new ShardId("bogus", "_na_", 1), test.getIndexSettings()); - assertEquals(indicesService.numPendingDeletes(test.index()), 2); + assertEquals(indicesService.numPendingDeletes(index), numPending); + assertTrue(indicesService.hasUncompletedPendingDeletes()); + + final boolean hasBogus = randomBoolean(); + if (hasBogus) { + indicesService.addPendingDelete(new ShardId(index, 0), indexSettings); + indicesService.addPendingDelete(new ShardId(index, 1), indexSettings); + indicesService.addPendingDelete(new ShardId("bogus", "_na_", 1), indexSettings); + assertEquals(indicesService.numPendingDeletes(index), numPending + 2); assertTrue(indicesService.hasUncompletedPendingDeletes()); - // shard lock released... we can now delete - indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); - assertEquals(indicesService.numPendingDeletes(test.index()), 0); - assertTrue(indicesService.hasUncompletedPendingDeletes()); // "bogus" index has not been removed } - assertAcked(client().admin().indices().prepareOpen("test").setTimeout(TimeValue.timeValueSeconds(1))); + assertAcked(client().admin().indices().prepareDelete("test")); + assertBusy(() -> { + try { + indicesService.processPendingDeletes(index, indexSettings, TimeValue.timeValueMillis(0)); + assertEquals(indicesService.numPendingDeletes(index), 0); + } catch (final Exception e) { + fail(e.getMessage()); + } + }); + assertThat(indicesService.hasUncompletedPendingDeletes(), equalTo(hasBogus)); // "bogus" index has not been removed + assertFalse(shardPath.exists()); } public void testVerifyIfIndexContentDeleted() throws Exception { From cf9a015122ace47c3530ec76e9d7d58d04799092 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 11 Feb 2019 13:09:41 +0100 Subject: [PATCH 07/20] Adapt testIndexCanChangeCustomDataPath for replicated closed indices (#38327) Relates to #33888 and #38024 --- .../index/shard/IndexShardIT.java | 96 ++++++++----------- 1 file changed, 42 insertions(+), 54 deletions(-) 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 fd08af610bbb4..5b861e7d52bd5 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -87,7 +87,6 @@ import java.io.UncheckedIOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -103,7 +102,9 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; +import java.util.stream.Stream; +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiLettersOfLength; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; @@ -111,7 +112,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; -import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.index.shard.IndexShardTestCase.getTranslog; @@ -277,76 +277,64 @@ public void testExpectedShardSizeIsPresent() throws InterruptedException { assertTrue(test > 0); } - // NORELEASE This test need to be adapted for replicated closed indices - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/33888") public void testIndexCanChangeCustomDataPath() throws Exception { - Environment env = getInstanceFromNode(Environment.class); - Path idxPath = env.sharedDataFile().resolve(randomAlphaOfLength(10)); - final String INDEX = "idx"; - Path startDir = idxPath.resolve("start-" + randomAlphaOfLength(10)); - Path endDir = idxPath.resolve("end-" + randomAlphaOfLength(10)); - logger.info("--> start dir: [{}]", startDir.toAbsolutePath().toString()); - logger.info("--> end dir: [{}]", endDir.toAbsolutePath().toString()); - // temp dirs are automatically created, but the end dir is what - // startDir is going to be renamed as, so it needs to be deleted - // otherwise we get all sorts of errors about the directory - // already existing - IOUtils.rm(endDir); - - Settings sb = Settings.builder() - .put(IndexMetaData.SETTING_DATA_PATH, startDir.toAbsolutePath().toString()) - .build(); - Settings sb2 = Settings.builder() - .put(IndexMetaData.SETTING_DATA_PATH, endDir.toAbsolutePath().toString()) - .build(); + final String index = "test-custom-data-path"; + final Path sharedDataPath = getInstanceFromNode(Environment.class).sharedDataFile().resolve(randomAsciiLettersOfLength(10)); + final Path indexDataPath = sharedDataPath.resolve("start-" + randomAsciiLettersOfLength(10)); - logger.info("--> creating an index with data_path [{}]", startDir.toAbsolutePath().toString()); - createIndex(INDEX, sb); - ensureGreen(INDEX); - client().prepareIndex(INDEX, "bar", "1").setSource("{}", XContentType.JSON).setRefreshPolicy(IMMEDIATE).get(); + logger.info("--> creating index [{}] with data_path [{}]", index, indexDataPath); + createIndex(index, Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, indexDataPath.toAbsolutePath().toString()).build()); + client().prepareIndex(index, "bar", "1").setSource("foo", "bar").setRefreshPolicy(IMMEDIATE).get(); + ensureGreen(index); - SearchResponse resp = client().prepareSearch(INDEX).setQuery(matchAllQuery()).get(); - assertThat("found the hit", resp.getHits().getTotalHits().value, equalTo(1L)); + assertHitCount(client().prepareSearch(index).setSize(0).get(), 1L); - logger.info("--> closing the index [{}]", INDEX); - client().admin().indices().prepareClose(INDEX).get(); + logger.info("--> closing the index [{}]", index); + assertAcked(client().admin().indices().prepareClose(index)); logger.info("--> index closed, re-opening..."); - client().admin().indices().prepareOpen(INDEX).get(); + assertAcked(client().admin().indices().prepareOpen(index)); logger.info("--> index re-opened"); - ensureGreen(INDEX); + ensureGreen(index); - resp = client().prepareSearch(INDEX).setQuery(matchAllQuery()).get(); - assertThat("found the hit", resp.getHits().getTotalHits().value, equalTo(1L)); + assertHitCount(client().prepareSearch(index).setSize(0).get(), 1L); // Now, try closing and changing the settings + logger.info("--> closing the index [{}] before updating data_path", index); + assertAcked(client().admin().indices().prepareClose(index)); - logger.info("--> closing the index [{}]", INDEX); - client().admin().indices().prepareClose(INDEX).get(); - - logger.info("--> moving data on disk [{}] to [{}]", startDir.getFileName(), endDir.getFileName()); - assert Files.exists(endDir) == false : "end directory should not exist!"; - Files.move(startDir, endDir, StandardCopyOption.REPLACE_EXISTING); + final Path newIndexDataPath = sharedDataPath.resolve("end-" + randomAlphaOfLength(10)); + IOUtils.rm(newIndexDataPath); - logger.info("--> updating settings..."); - client().admin().indices().prepareUpdateSettings(INDEX) - .setSettings(sb2) - .setIndicesOptions(IndicesOptions.fromOptions(true, false, true, true)) - .get(); + logger.info("--> copying data on disk from [{}] to [{}]", indexDataPath, newIndexDataPath); + assert Files.exists(newIndexDataPath) == false : "new index data path directory should not exist!"; + try (Stream stream = Files.walk(indexDataPath)) { + stream.forEach(path -> { + try { + if (path.endsWith(".lock") == false) { + Files.copy(path, newIndexDataPath.resolve(indexDataPath.relativize(path))); + } + } catch (final Exception e) { + logger.error("Failed to copy data path directory", e); + fail(); + } + }); + } - assert Files.exists(startDir) == false : "start dir shouldn't exist"; + logger.info("--> updating data_path to [{}] for index [{}]", newIndexDataPath, index); + assertAcked(client().admin().indices().prepareUpdateSettings(index) + .setSettings(Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, newIndexDataPath.toAbsolutePath().toString()).build()) + .setIndicesOptions(IndicesOptions.fromOptions(true, false, true, true))); logger.info("--> settings updated and files moved, re-opening index"); - client().admin().indices().prepareOpen(INDEX).get(); + assertAcked(client().admin().indices().prepareOpen(index)); logger.info("--> index re-opened"); - ensureGreen(INDEX); + ensureGreen(index); - resp = client().prepareSearch(INDEX).setQuery(matchAllQuery()).get(); - assertThat("found the hit", resp.getHits().getTotalHits().value, equalTo(1L)); + assertHitCount(client().prepareSearch(index).setSize(0).get(), 1L); - assertAcked(client().admin().indices().prepareDelete(INDEX)); + assertAcked(client().admin().indices().prepareDelete(index)); assertAllIndicesRemovedAndDeletionCompleted(Collections.singleton(getInstanceFromNode(IndicesService.class))); - assertPathHasBeenCleared(startDir.toAbsolutePath()); - assertPathHasBeenCleared(endDir.toAbsolutePath()); + assertPathHasBeenCleared(newIndexDataPath.toAbsolutePath()); } public void testMaybeFlush() throws Exception { From e845b0ae746b42516fe6e6870742b9a0424806bc Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 11 Feb 2019 13:38:16 +0100 Subject: [PATCH 08/20] Do not schedule Refresh/Translog/GlobalCheckpoint tasks for closed indices (#38329) Replicated closed indices do not need to be refreshed, neither they need their translogs or global checkpoint to be fsync. This pull request changes how `BaseAsyncTask` tasks are rescheduled in `IndexService` instances so that the tasks are rescheduled only when the index is opened. Relates to #33888 --- .../org/elasticsearch/index/IndexService.java | 11 ++- .../index/IndexServiceTests.java | 80 ++++++++++++++++++- 2 files changed, 84 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 1b1784495e685..8bbf048fe1246 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -672,7 +672,7 @@ public synchronized void updateMetaData(final IndexMetaData currentIndexMetaData // once we change the refresh interval we schedule yet another refresh // to ensure we are in a clean and predictable state. // it doesn't matter if we move from or to -1 in both cases we want - // docs to become visible immediately. This also flushes all pending indexing / search reqeusts + // docs to become visible immediately. This also flushes all pending indexing / search requests // that are waiting for a refresh. threadPool.executor(ThreadPool.Names.REFRESH).execute(new AbstractRunnable() { @Override @@ -829,17 +829,20 @@ private void sync(final Consumer sync, final String source) { } abstract static class BaseAsyncTask extends AbstractAsyncTask { + protected final IndexService indexService; - BaseAsyncTask(IndexService indexService, TimeValue interval) { + BaseAsyncTask(final IndexService indexService, final TimeValue interval) { super(indexService.logger, indexService.threadPool, interval, true); this.indexService = indexService; rescheduleIfNecessary(); } + @Override protected boolean mustReschedule() { - // don't re-schedule if its closed or if we don't have a single shard here..., we are done - return indexService.closed.get() == false; + // don't re-schedule if the IndexService instance is closed or if the index is closed + return indexService.closed.get() == false + && indexService.indexSettings.getIndexMetaData().getState() == IndexMetaData.State.OPEN; } } diff --git a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java index a47d4db2a2579..e5e554818c020 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexServiceTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.InternalSettingsPlugin; @@ -47,6 +48,7 @@ import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.test.InternalSettingsPlugin.TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.core.IsEqual.equalTo; /** Unit test(s) for IndexService */ @@ -109,7 +111,6 @@ protected String getThreadPool() { latch2.get().countDown(); assertEquals(2, count.get()); - task = new IndexService.BaseAsyncTask(indexService, TimeValue.timeValueMillis(1000000)) { @Override protected void runInternal() { @@ -117,6 +118,34 @@ protected void runInternal() { } }; assertTrue(task.mustReschedule()); + + // now close the index + final Index index = indexService.index(); + assertAcked(client().admin().indices().prepareClose(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + + final IndexService closedIndexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(indexService, closedIndexService); + assertFalse(task.mustReschedule()); + assertFalse(task.isClosed()); + assertEquals(1000000, task.getInterval().millis()); + + // now reopen the index + assertAcked(client().admin().indices().prepareOpen(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(closedIndexService, indexService); + + task = new IndexService.BaseAsyncTask(indexService, TimeValue.timeValueMillis(100000)) { + @Override + protected void runInternal() { + + } + }; + assertTrue(task.mustReschedule()); + assertFalse(task.isClosed()); + assertTrue(task.isScheduled()); + indexService.close("simon says", false); assertFalse("no shards left", task.mustReschedule()); assertTrue(task.isScheduled()); @@ -124,7 +153,7 @@ protected void runInternal() { assertFalse(task.isScheduled()); } - public void testRefreshTaskIsUpdated() throws IOException { + public void testRefreshTaskIsUpdated() throws Exception { IndexService indexService = createIndex("test", Settings.EMPTY); IndexService.AsyncRefreshTask refreshTask = indexService.getRefreshTask(); assertEquals(1000, refreshTask.getInterval().millis()); @@ -167,12 +196,35 @@ public void testRefreshTaskIsUpdated() throws IOException { assertTrue(refreshTask.isScheduled()); assertFalse(refreshTask.isClosed()); assertEquals(200, refreshTask.getInterval().millis()); + + // now close the index + final Index index = indexService.index(); + assertAcked(client().admin().indices().prepareClose(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + + final IndexService closedIndexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(indexService, closedIndexService); + assertNotSame(refreshTask, closedIndexService.getRefreshTask()); + assertFalse(closedIndexService.getRefreshTask().mustReschedule()); + assertFalse(closedIndexService.getRefreshTask().isClosed()); + assertEquals(200, closedIndexService.getRefreshTask().getInterval().millis()); + + // now reopen the index + assertAcked(client().admin().indices().prepareOpen(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(closedIndexService, indexService); + refreshTask = indexService.getRefreshTask(); + assertTrue(indexService.getRefreshTask().mustReschedule()); + assertTrue(refreshTask.isScheduled()); + assertFalse(refreshTask.isClosed()); + indexService.close("simon says", false); assertFalse(refreshTask.isScheduled()); assertTrue(refreshTask.isClosed()); } - public void testFsyncTaskIsRunning() throws IOException { + public void testFsyncTaskIsRunning() throws Exception { Settings settings = Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING.getKey(), Translog.Durability.ASYNC).build(); IndexService indexService = createIndex("test", settings); @@ -182,6 +234,28 @@ public void testFsyncTaskIsRunning() throws IOException { assertTrue(fsyncTask.mustReschedule()); assertTrue(fsyncTask.isScheduled()); + // now close the index + final Index index = indexService.index(); + assertAcked(client().admin().indices().prepareClose(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + + final IndexService closedIndexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(indexService, closedIndexService); + assertNotSame(fsyncTask, closedIndexService.getFsyncTask()); + assertFalse(closedIndexService.getFsyncTask().mustReschedule()); + assertFalse(closedIndexService.getFsyncTask().isClosed()); + assertEquals(5000, closedIndexService.getFsyncTask().getInterval().millis()); + + // now reopen the index + assertAcked(client().admin().indices().prepareOpen(index.getName())); + awaitBusy(() -> getInstanceFromNode(IndicesService.class).hasIndex(index)); + indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(index); + assertNotSame(closedIndexService, indexService); + fsyncTask = indexService.getFsyncTask(); + assertTrue(indexService.getRefreshTask().mustReschedule()); + assertTrue(fsyncTask.isScheduled()); + assertFalse(fsyncTask.isClosed()); + indexService.close("simon says", false); assertFalse(fsyncTask.isScheduled()); assertTrue(fsyncTask.isClosed()); From 00f1828ca894cb4c8f644b3dd66fde852d064247 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 15 Feb 2019 16:06:35 +0100 Subject: [PATCH 09/20] Mute CloseFollowerIndexIT.testCloseAndReopenFollowerIndex() --- .../java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java index 0551d30c2e73a..bd9f1a13bdb4d 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java @@ -31,6 +31,7 @@ public class CloseFollowerIndexIT extends CcrIntegTestCase { + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/38767") public void testCloseAndReopenFollowerIndex() throws Exception { final String leaderIndexSettings = getIndexSettings(1, 1, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); From c484c66f3f7180e97de7a413a2cee80b9baa62b7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Feb 2019 20:20:55 +0100 Subject: [PATCH 10/20] Remove index routing table of closed indices in mixed versions clusters (#38955) This pull request removes the legacy way of closing indices (aka "direct close") in mixed versions clusters, since this backward compatibility logic is not required anymore on master/8.0.0. It also changes the closing logic so that routing tables of closed indices are removed when the cluster contains a node in version < 8.0. Relates #33888 --- .../metadata/MetaDataIndexStateService.java | 31 ++++---- .../MetaDataIndexStateServiceTests.java | 72 +++++++++++++------ 2 files changed, 66 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index 3e9143320c53c..ca58b49e51860 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -223,10 +223,6 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final Map blockedIndices, final Map results) { + + // Remove the index routing table of closed indices if the cluster is in a mixed version + // that does not support the replication of closed indices + final boolean removeRoutingTable = currentState.nodes().getMinNodeVersion().before(Version.V_8_0_0); + final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable()); @@ -413,7 +406,11 @@ static ClusterState closeRoutingTable(final ClusterState currentState, metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID); blocks.addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); - routingTable.addAsFromOpenToClose(metadata.getSafe(index)); + if (removeRoutingTable) { + routingTable.remove(index.getName()); + } else { + routingTable.addAsFromOpenToClose(metadata.getSafe(index)); + } closedIndices.add(index.getName()); } catch (final IndexNotFoundException e) { logger.debug("index {} has been deleted since it was blocked before closing, ignoring", index); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index 4108c542d0cc5..df07982f445ba 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -115,6 +115,57 @@ public void testCloseRoutingTable() { } } + public void testCloseRoutingTableRemovesRoutingTable() { + final Set nonBlockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); + final Map results = new HashMap<>(); + final ClusterBlock closingBlock = MetaDataIndexStateService.createIndexClosingBlock(); + + ClusterState state = ClusterState.builder(new ClusterName("testCloseRoutingTableRemovesRoutingTable")).build(); + for (int i = 0; i < randomIntBetween(1, 25); i++) { + final String indexName = "index-" + i; + + if (randomBoolean()) { + state = addOpenedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state); + nonBlockedIndices.add(state.metaData().index(indexName).getIndex()); + } else { + state = addBlockedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state, closingBlock); + blockedIndices.put(state.metaData().index(indexName).getIndex(), closingBlock); + results.put(state.metaData().index(indexName).getIndex(), new AcknowledgedResponse(randomBoolean())); + } + } + + state = ClusterState.builder(state) + .nodes(DiscoveryNodes.builder(state.nodes()) + .add(new DiscoveryNode("old_node", buildNewFakeTransportAddress(), emptyMap(), + new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.V_7_1_0)) + .add(new DiscoveryNode("new_node", buildNewFakeTransportAddress(), emptyMap(), + new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.V_8_0_0))) + .build(); + + state = MetaDataIndexStateService.closeRoutingTable(state, blockedIndices, results); + assertThat(state.metaData().indices().size(), equalTo(nonBlockedIndices.size() + blockedIndices.size())); + + for (Index nonBlockedIndex : nonBlockedIndices) { + assertIsOpened(nonBlockedIndex.getName(), state); + assertThat(state.blocks().hasIndexBlockWithId(nonBlockedIndex.getName(), INDEX_CLOSED_BLOCK_ID), is(false)); + } + for (Index blockedIndex : blockedIndices.keySet()) { + if (results.get(blockedIndex).isAcknowledged()) { + assertThat(state.metaData().index(blockedIndex).getState(), is(IndexMetaData.State.CLOSE)); + assertThat(state.blocks().hasIndexBlock(blockedIndex.getName(), MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + assertThat("Index must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", + state.blocks().indices().getOrDefault(blockedIndex.getName(), emptySet()).stream() + .filter(clusterBlock -> clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); + assertThat("Index routing table should have been removed when closing the index on mixed cluster version", + state.routingTable().index(blockedIndex), nullValue()); + } else { + assertIsOpened(blockedIndex.getName(), state); + assertThat(state.blocks().hasIndexBlock(blockedIndex.getName(), closingBlock), is(true)); + } + } + } + public void testAddIndexClosedBlocks() { final ClusterState initialState = ClusterState.builder(new ClusterName("testAddIndexClosedBlocks")).build(); { @@ -191,14 +242,6 @@ public void testAddIndexClosedBlocks() { ClusterState state = addOpenedIndex("index-1", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); state = addOpenedIndex("index-2", randomIntBetween(1, 3), randomIntBetween(0, 3), state); state = addOpenedIndex("index-3", randomIntBetween(1, 3), randomIntBetween(0, 3), state); - final boolean mixedVersions = randomBoolean(); - if (mixedVersions) { - state = ClusterState.builder(state) - .nodes(DiscoveryNodes.builder(state.nodes()) - .add(new DiscoveryNode("old_node", buildNewFakeTransportAddress(), emptyMap(), - new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.V_6_0_0))) - .build(); - } Index index1 = state.metaData().index("index-1").getIndex(); Index index2 = state.metaData().index("index-2").getIndex(); @@ -210,18 +253,7 @@ public void testAddIndexClosedBlocks() { for (Index index : indices) { assertTrue(blockedIndices.containsKey(index)); - if (mixedVersions) { - assertThat(updatedState.metaData().index(index).getState(), is(IndexMetaData.State.CLOSE)); - assertTrue(updatedState.blocks().hasIndexBlock(index.getName(), MetaDataIndexStateService.INDEX_CLOSED_BLOCK)); - assertThat("Index " + index + " must have only 1 block with id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID, - updatedState.blocks().indices().getOrDefault(index.getName(), emptySet()).stream().filter(clusterBlock -> - clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); - - final IndexRoutingTable indexRoutingTable = updatedState.routingTable().index(index); - assertThat(indexRoutingTable, nullValue()); - } else { - assertHasBlock(index.getName(), updatedState, blockedIndices.get(index)); - } + assertHasBlock(index.getName(), updatedState, blockedIndices.get(index)); } } } From b756f6cdb021581c0077911ca6d87e0bfa769e2f Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Feb 2019 21:59:42 +0100 Subject: [PATCH 11/20] Test the Cluster Shard Allocation Explain API with closed indices (#38631) This pull request modifies the `ClusterAllocationExplainIT` test suite so that it always runs the tests with opened and closed indices. The only test that was not adapted for closed indices is `testAllocationFilteringOnIndexCreation` because we don't allow to directly create indices in the closed state. Relates to #33888 --- .../cluster.allocation_explain/10_basic.yml | 44 ++++++++++ .../ClusterAllocationExplainIT.java | 83 +++++++++++++------ 2 files changed, 101 insertions(+), 26 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.allocation_explain/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.allocation_explain/10_basic.yml index 7dbc57dac8b56..510a872f2b5b9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.allocation_explain/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.allocation_explain/10_basic.yml @@ -53,3 +53,47 @@ - match: { primary: false } - is_true: cluster_info - is_true: can_allocate + + +--- +"Cluster shard allocation explanation test with a closed index": + - skip: + version: " - 7.99.99" + reason: closed indices are replicated starting version 8.0.0 + + - do: + indices.create: + index: test_closed + body: { "settings": { "index.number_of_shards": 1, "index.number_of_replicas": 0 } } + + - match: { acknowledged: true } + + - do: + cluster.health: + index: test_closed + wait_for_status: green + + - do: + indices.close: + index: test_closed + + - match: { acknowledged: true } + + - do: + cluster.health: + index: test_closed + wait_for_status: green + + - do: + cluster.allocation_explain: + body: { "index": "test_closed", "shard": 0, "primary": true } + + - match: { current_state: "started" } + - is_true: current_node.id + - match: { index: "test_closed" } + - match: { shard: 0 } + - match: { primary: true } + - is_true: can_remain_on_current_node + - is_true: can_rebalance_cluster + - is_true: can_rebalance_to_other_node + - is_true: rebalance_explanation diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java index e38fc64c8e3ab..941ad3c658aba 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java @@ -19,9 +19,12 @@ package org.elasticsearch.action.admin.cluster.allocation; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ClusterInfo; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.UnassignedInfo; @@ -32,8 +35,10 @@ import org.elasticsearch.cluster.routing.allocation.MoveDecision; import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult; import org.elasticsearch.cluster.routing.allocation.decider.Decision; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -52,6 +57,7 @@ import java.util.Map; import java.util.Set; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -70,8 +76,7 @@ public void testUnassignedPrimaryWithExistingIndex() throws Exception { logger.info("--> starting 2 nodes"); internalCluster().startNodes(2); - logger.info("--> creating an index with 1 primary, 0 replicas"); - createIndexAndIndexData(1, 0); + prepareIndex(1, 0); logger.info("--> stopping the node with the primary"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName())); @@ -149,8 +154,7 @@ public void testUnassignedReplicaDelayedAllocation() throws Exception { logger.info("--> starting 3 nodes"); internalCluster().startNodes(3); - logger.info("--> creating an index with 1 primary, 1 replica"); - createIndexAndIndexData(1, 1); + prepareIndex(1, 1); logger.info("--> stopping the node with the replica"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(replicaNode().getName())); ensureStableCluster(2); @@ -268,8 +272,7 @@ public void testUnassignedReplicaWithPriorCopy() throws Exception { logger.info("--> starting 3 nodes"); List nodes = internalCluster().startNodes(3); - logger.info("--> creating an index with 1 primary and 1 replica"); - createIndexAndIndexData(1, 1); + prepareIndex(1, 1); String primaryNodeName = primaryNodeName(); nodes.remove(primaryNodeName); @@ -390,7 +393,8 @@ public void testAllocationFilteringOnIndexCreation() throws Exception { internalCluster().startNodes(2); logger.info("--> creating an index with 1 primary, 0 replicas, with allocation filtering so the primary can't be assigned"); - createIndexAndIndexData(1, 0, Settings.builder().put("index.routing.allocation.include._name", "non_existent_node").build(), + prepareIndex(IndexMetaData.State.OPEN, 1, 0, + Settings.builder().put("index.routing.allocation.include._name", "non_existent_node").build(), ActiveShardCount.NONE); boolean includeYesDecisions = randomBoolean(); @@ -481,8 +485,7 @@ public void testAllocationFilteringPreventsShardMove() throws Exception { logger.info("--> starting 2 nodes"); internalCluster().startNodes(2); - logger.info("--> creating an index with 1 primary and 0 replicas"); - createIndexAndIndexData(1, 0); + prepareIndex(1, 0); logger.info("--> setting up allocation filtering to prevent allocation to both nodes"); client().admin().indices().prepareUpdateSettings("idx").setSettings( @@ -591,8 +594,7 @@ public void testRebalancingNotAllowed() throws Exception { internalCluster().startNode(); ensureStableCluster(1); - logger.info("--> creating an index with 5 shards, all allocated to the single node"); - createIndexAndIndexData(5, 0); + prepareIndex(5, 0); logger.info("--> disabling rebalancing on the index"); client().admin().indices().prepareUpdateSettings("idx").setSettings( @@ -704,8 +706,7 @@ public void testWorseBalance() throws Exception { internalCluster().startNode(); ensureStableCluster(1); - logger.info("--> creating an index with 5 shards, all allocated to the single node"); - createIndexAndIndexData(5, 0); + prepareIndex(5, 0); logger.info("--> setting balancing threshold really high, so it won't be met"); client().admin().cluster().prepareUpdateSettings().setTransientSettings( @@ -808,8 +809,7 @@ public void testBetterBalanceButCannotAllocate() throws Exception { String firstNode = internalCluster().startNode(); ensureStableCluster(1); - logger.info("--> creating an index with 5 shards, all allocated to the single node"); - createIndexAndIndexData(5, 0); + prepareIndex(5, 0); logger.info("--> setting up allocation filtering to only allow allocation to the current node"); client().admin().indices().prepareUpdateSettings("idx").setSettings( @@ -918,9 +918,9 @@ public void testAssignedReplicaOnSpecificNode() throws Exception { logger.info("--> starting 3 nodes"); List nodes = internalCluster().startNodes(3); - logger.info("--> creating an index with 1 primary and 2 replicas"); String excludedNode = nodes.get(randomIntBetween(0, 2)); - createIndexAndIndexData(1, 2, Settings.builder().put("index.routing.allocation.exclude._name", excludedNode).build(), + prepareIndex(randomIndexState(), 1, 2, + Settings.builder().put("index.routing.allocation.exclude._name", excludedNode).build(), ActiveShardCount.from(2)); boolean includeYesDecisions = randomBoolean(); @@ -1019,8 +1019,7 @@ public void testCannotAllocateStaleReplicaExplanation() throws Exception { final String replicaNode = internalCluster().startNode(); final String primaryNode = internalCluster().startNode(); - logger.info("--> creating an index with 1 primary and 1 replica"); - createIndexAndIndexData(1, 1, + prepareIndex(IndexMetaData.State.OPEN, 1, 1, Settings.builder() .put("index.routing.allocation.include._name", primaryNode) .put("index.routing.allocation.exclude._name", masterNode) @@ -1037,8 +1036,22 @@ public void testCannotAllocateStaleReplicaExplanation() throws Exception { logger.info("--> stop node with the replica shard"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(replicaNode)); - logger.info("--> index more data, now the replica is stale"); - indexData(); + final IndexMetaData.State indexState = randomIndexState(); + if (indexState == IndexMetaData.State.OPEN) { + logger.info("--> index more data, now the replica is stale"); + indexData(); + } else { + logger.info("--> close the index, now the replica is stale"); + assertAcked(client().admin().indices().prepareClose("idx")); + + final ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth("idx") + .setTimeout(TimeValue.timeValueSeconds(30)) + .setWaitForActiveShards(ActiveShardCount.ONE) + .setWaitForNoInitializingShards(true) + .setWaitForEvents(Priority.LANGUID) + .get(); + assertThat(clusterHealthResponse.getStatus().value(), lessThanOrEqualTo(ClusterHealthStatus.YELLOW.value())); + } logger.info("--> stop the node with the primary"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNode)); @@ -1147,21 +1160,39 @@ private ClusterAllocationExplanation runExplain(boolean primary, String nodeId, return explanation; } - private void createIndexAndIndexData(int numPrimaries, int numReplicas) { - createIndexAndIndexData(numPrimaries, numReplicas, Settings.EMPTY, ActiveShardCount.ALL); + private void prepareIndex(final int numPrimaries, final int numReplicas) { + prepareIndex(randomIndexState(), numPrimaries, numReplicas, Settings.EMPTY, ActiveShardCount.ALL); } - private void createIndexAndIndexData(int numPrimaries, int numReplicas, Settings settings, ActiveShardCount activeShardCount) { - client().admin().indices().prepareCreate("idx") + private void prepareIndex(final IndexMetaData.State state, final int numPrimaries, final int numReplicas, + final Settings settings, final ActiveShardCount activeShardCount) { + + logger.info("--> creating a {} index with {} primary, {} replicas", state, numPrimaries, numReplicas); + assertAcked(client().admin().indices().prepareCreate("idx") .setSettings(Settings.builder() .put("index.number_of_shards", numPrimaries) .put("index.number_of_replicas", numReplicas) .put(settings)) .setWaitForActiveShards(activeShardCount) - .get(); + .get()); + if (activeShardCount != ActiveShardCount.NONE) { indexData(); } + if (state == IndexMetaData.State.CLOSE) { + assertAcked(client().admin().indices().prepareClose("idx")); + + final ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth("idx") + .setTimeout(TimeValue.timeValueSeconds(30)) + .setWaitForActiveShards(activeShardCount) + .setWaitForEvents(Priority.LANGUID) + .get(); + assertThat(clusterHealthResponse.getStatus().value(), lessThanOrEqualTo(ClusterHealthStatus.YELLOW.value())); + } + } + + private static IndexMetaData.State randomIndexState() { + return randomFrom(IndexMetaData.State.values()); } private void indexData() { From 0519016d996cb89b5fbb85fdf5c12cd8aa57051b Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 21 Feb 2019 09:53:27 +0100 Subject: [PATCH 12/20] Add replica to primary promotion test for closed indices (#39110) This commit adds a simple test which verifies that a replica can be promoted as a primary when the index is closed. Relates to #33888 --- .../recovery/ReplicaToPrimaryPromotionIT.java | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 server/src/test/java/org/elasticsearch/indices/recovery/ReplicaToPrimaryPromotionIT.java diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/ReplicaToPrimaryPromotionIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/ReplicaToPrimaryPromotionIT.java new file mode 100644 index 0000000000000..126c4df7928cd --- /dev/null +++ b/server/src/test/java/org/elasticsearch/indices/recovery/ReplicaToPrimaryPromotionIT.java @@ -0,0 +1,87 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.indices.recovery; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.test.BackgroundIndexer; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalTestCluster; + +import java.util.Locale; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.is; + +@ESIntegTestCase.ClusterScope(numDataNodes = 2) +public class ReplicaToPrimaryPromotionIT extends ESIntegTestCase { + + @Override + protected int numberOfReplicas() { + return 1; + } + + public void testPromoteReplicaToPrimary() throws Exception { + final String indexName = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); + createIndex(indexName); + + final int numOfDocs = scaledRandomIntBetween(0, 200); + try (BackgroundIndexer indexer = new BackgroundIndexer(indexName, "_doc", client(), numOfDocs)) { + waitForDocs(numOfDocs, indexer); + } + refresh(indexName); + + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), numOfDocs); + ensureGreen(indexName); + + // sometimes test with a closed index + final IndexMetaData.State indexState = randomFrom(IndexMetaData.State.OPEN, IndexMetaData.State.CLOSE); + if (indexState == IndexMetaData.State.CLOSE) { + assertAcked(client().admin().indices().prepareClose(indexName)); + ensureGreen(indexName); + } + + // pick up a data node that contains a random primary shard + ClusterState state = client(internalCluster().getMasterName()).admin().cluster().prepareState().get().getState(); + final int numShards = state.metaData().index(indexName).getNumberOfShards(); + final ShardRouting primaryShard = state.routingTable().index(indexName).shard(randomIntBetween(0, numShards - 1)).primaryShard(); + final DiscoveryNode randomNode = state.nodes().resolveNode(primaryShard.currentNodeId()); + + // stop the random data node, all remaining shards are promoted to primaries + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(randomNode.getName())); + ensureYellowAndNoInitializingShards(indexName); + + state = client(internalCluster().getMasterName()).admin().cluster().prepareState().get().getState(); + for (IndexShardRoutingTable shardRoutingTable : state.routingTable().index(indexName)) { + for (ShardRouting shardRouting : shardRoutingTable.activeShards()) { + assertThat(shardRouting + " should be promoted as a primary", shardRouting.primary(), is(true)); + } + } + + if (indexState == IndexMetaData.State.CLOSE) { + assertAcked(client().admin().indices().prepareOpen(indexName)); + ensureYellowAndNoInitializingShards(indexName); + } + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), numOfDocs); + } +} From 4fd1bb290aa0ffeaf9f8845695701a26729308d2 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 22 Feb 2019 13:28:59 +0100 Subject: [PATCH 13/20] Adapt more tests suites to closed indices (#39186) * Adapt more tests suites to closed indices Similarly to #38631, this pull request modifies multiple test suites so that they runs the tests with opened or closed indices. The suites are testing: - shard allocation filtering - shard allocation awereness - Reroute API Relates to #33888 --- .../allocation/AwarenessAllocationIT.java | 109 ++++++++++++++---- .../cluster/allocation/ClusterRerouteIT.java | 70 +++++++---- .../allocation/FilteringAllocationIT.java | 57 ++++++--- .../allocation/SimpleAllocationIT.java | 18 +-- 4 files changed, 182 insertions(+), 72 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/allocation/AwarenessAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/allocation/AwarenessAllocationIT.java index 083c914b37052..edcf4446dc2bf 100644 --- a/server/src/test/java/org/elasticsearch/cluster/allocation/AwarenessAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/allocation/AwarenessAllocationIT.java @@ -24,6 +24,8 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexMetaData.State; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -33,9 +35,11 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; +import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -54,7 +58,6 @@ public void testSimpleAwareness() throws Exception { .put("cluster.routing.allocation.awareness.attributes", "rack_id") .build(); - logger.info("--> starting 2 nodes on the same rack"); internalCluster().startNodes(2, Settings.builder().put(commonSettings).put("node.attr.rack_id", "rack_1").build()); @@ -68,6 +71,9 @@ public void testSimpleAwareness() throws Exception { ensureGreen(); + final List indicesToClose = randomSubsetOf(Arrays.asList("test1", "test2")); + indicesToClose.forEach(indexToClose -> assertAcked(client().admin().indices().prepareClose(indexToClose).get())); + logger.info("--> starting 1 node on a different rack"); final String node3 = internalCluster().startNode(Settings.builder().put(commonSettings).put("node.attr.rack_id", "rack_2").build()); @@ -75,14 +81,23 @@ public void testSimpleAwareness() throws Exception { assertThat(awaitBusy( () -> { logger.info("--> waiting for no relocation"); - ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) - .setWaitForGreenStatus().setWaitForNodes("3").setWaitForNoRelocatingShards(true).get(); + ClusterHealthResponse clusterHealth = client().admin().cluster().prepareHealth() + .setIndices("test1", "test2") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("3") + .setWaitForNoRelocatingShards(true) + .get(); if (clusterHealth.isTimedOut()) { return false; } logger.info("--> checking current state"); ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); + // check that closed indices are effectively closed + if (indicesToClose.stream().anyMatch(index -> clusterState.metaData().index(index).getState() != State.CLOSE)) { + return false; + } // verify that we have all the primaries on node3 ObjectIntHashMap counts = new ObjectIntHashMap<>(); for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) { @@ -99,7 +114,7 @@ public void testSimpleAwareness() throws Exception { ), equalTo(true)); } - public void testAwarenessZones() throws Exception { + public void testAwarenessZones() { Settings commonSettings = Settings.builder() .put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING.getKey() + "zone.values", "a,b") .put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.getKey(), "zone") @@ -121,12 +136,20 @@ public void testAwarenessZones() throws Exception { ClusterHealthResponse health = client().admin().cluster().prepareHealth().setWaitForNodes("4").execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder().put("index.number_of_shards", 5) - .put("index.number_of_replicas", 1)).execute().actionGet(); + createIndex("test", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .build()); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test")); + } logger.info("--> waiting for shards to be allocated"); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() .setWaitForNoRelocatingShards(true).execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); @@ -146,7 +169,7 @@ public void testAwarenessZones() throws Exception { assertThat(counts.get(B_0), anyOf(equalTo(2),equalTo(3))); } - public void testAwarenessZonesIncrementalNodes() throws Exception { + public void testAwarenessZonesIncrementalNodes() { Settings commonSettings = Settings.builder() .put("cluster.routing.allocation.awareness.force.zone.values", "a,b") .put("cluster.routing.allocation.awareness.attributes", "zone") @@ -159,11 +182,23 @@ public void testAwarenessZonesIncrementalNodes() throws Exception { ); String A_0 = nodes.get(0); String B_0 = nodes.get(1); - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder().put("index.number_of_shards", 5) - .put("index.number_of_replicas", 1)).execute().actionGet(); - ClusterHealthResponse health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) - .setWaitForGreenStatus().setWaitForNodes("2").setWaitForNoRelocatingShards(true).execute().actionGet(); + + createIndex("test", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .build()); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test")); + } + + ClusterHealthResponse health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("2") + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); ObjectIntHashMap counts = new ObjectIntHashMap<>(); @@ -180,12 +215,22 @@ public void testAwarenessZonesIncrementalNodes() throws Exception { logger.info("--> starting another node in zone 'b'"); String B_1 = internalCluster().startNode(Settings.builder().put(commonSettings).put("node.attr.zone", "b").build()); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() - .setWaitForNodes("3").execute().actionGet(); + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("3") + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); client().admin().cluster().prepareReroute().get(); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() - .setWaitForNodes("3").setWaitForActiveShards(10).setWaitForNoRelocatingShards(true).execute().actionGet(); + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("3") + .setWaitForActiveShards(10) + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); @@ -204,12 +249,22 @@ public void testAwarenessZonesIncrementalNodes() throws Exception { assertThat(counts.get(B_1), equalTo(2)); String noZoneNode = internalCluster().startNode(); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() - .setWaitForNodes("4").execute().actionGet(); + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("4") + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); client().admin().cluster().prepareReroute().get(); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() - .setWaitForNodes("4").setWaitForActiveShards(10).setWaitForNoRelocatingShards(true).execute().actionGet(); + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("4") + .setWaitForActiveShards(10) + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); @@ -231,8 +286,14 @@ public void testAwarenessZonesIncrementalNodes() throws Exception { client().admin().cluster().prepareUpdateSettings() .setTransientSettings(Settings.builder().put("cluster.routing.allocation.awareness.attributes", "").build()).get(); - health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() - .setWaitForNodes("4").setWaitForActiveShards(10).setWaitForNoRelocatingShards(true).execute().actionGet(); + health = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForGreenStatus() + .setWaitForNodes("4") + .setWaitForActiveShards(10) + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(health.isTimedOut(), equalTo(false)); clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); diff --git a/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java b/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java index 71c9f5a15ba4d..95cb09df0f68c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; import org.elasticsearch.action.admin.cluster.reroute.TransportClusterRerouteAction; @@ -34,6 +33,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.RerouteExplanation; import org.elasticsearch.cluster.routing.allocation.RoutingExplanations; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; @@ -48,6 +48,7 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -102,6 +103,10 @@ private void rerouteWithCommands(Settings commonSettings) throws Exception { .setSettings(Settings.builder().put("index.number_of_shards", 1)) .execute().actionGet(); + if (randomBoolean()) { + client().admin().indices().prepareClose("test").get(); + } + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(2)); @@ -128,8 +133,11 @@ private void rerouteWithCommands(Settings commonSettings) throws Exception { assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.INITIALIZING)); - ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) - .setWaitForYellowStatus().execute().actionGet(); + ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForYellowStatus() + .execute().actionGet(); assertThat(healthResponse.isTimedOut(), equalTo(false)); logger.info("--> get the state, verify shard 1 primary allocated"); @@ -149,9 +157,12 @@ private void rerouteWithCommands(Settings commonSettings) throws Exception { assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_2).getId()).iterator().next().state(), equalTo(ShardRoutingState.INITIALIZING)); - - healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus() - .setWaitForNoRelocatingShards(true).execute().actionGet(); + healthResponse = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForYellowStatus() + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(healthResponse.isTimedOut(), equalTo(false)); logger.info("--> get the state, verify shard 1 primary moved from node1 to node2"); @@ -193,11 +204,15 @@ public void testDelayWithALargeAmountOfShards() throws Exception { logger.info("--> create indices"); for (int i = 0; i < 25; i++) { - client().admin().indices().prepareCreate("test" + i) - .setSettings(Settings.builder() - .put("index.number_of_shards", 5).put("index.number_of_replicas", 1) - .put("index.unassigned.node_left.delayed_timeout", randomIntBetween(250, 1000) + "ms")) - .execute().actionGet(); + final String indexName = "test" + i; + createIndex(indexName, Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), randomIntBetween(250, 1000) + "ms") + .build()); + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose(indexName)); + } } ensureGreen(TimeValue.timeValueMinutes(1)); @@ -294,10 +309,14 @@ public void testRerouteExplain() { assertThat(healthResponse.isTimedOut(), equalTo(false)); logger.info("--> create an index with 1 shard"); - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0)) - .execute().actionGet(); + createIndex("test", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .build()); + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test")); + } ensureGreen("test"); logger.info("--> disable allocation"); @@ -403,12 +422,18 @@ public void testMessageLogging() throws Exception{ Loggers.removeAppender(actionLogger, allocateMockLog); } - public void testClusterRerouteWithBlocks() throws Exception { + public void testClusterRerouteWithBlocks() { List nodesIds = internalCluster().startNodes(2); logger.info("--> create an index with 1 shard and 0 replicas"); - assertAcked(prepareCreate("test-blocks").setSettings(Settings.builder().put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0))); + createIndex("test-blocks", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .build()); + + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test-blocks")); + } ensureGreen("test-blocks"); logger.info("--> check that the index has 1 shard"); @@ -432,11 +457,14 @@ public void testClusterRerouteWithBlocks() throws Exception { SETTING_READ_ONLY_ALLOW_DELETE)) { try { enableIndexBlock("test-blocks", blockSetting); - assertAcked(client().admin().cluster().prepareReroute().add(new MoveAllocationCommand("test-blocks", 0, - nodesIds.get(toggle % 2), nodesIds.get(++toggle % 2)))); + assertAcked(client().admin().cluster().prepareReroute() + .add(new MoveAllocationCommand("test-blocks", 0, nodesIds.get(toggle % 2), nodesIds.get(++toggle % 2)))); - ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth().setWaitForYellowStatus() - .setWaitForNoRelocatingShards(true).execute().actionGet(); + ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth() + .setIndices("test-blocks") + .setWaitForYellowStatus() + .setWaitForNoRelocatingShards(true) + .execute().actionGet(); assertThat(healthResponse.isTimedOut(), equalTo(false)); } finally { disableIndexBlock("test-blocks", blockSetting); diff --git a/server/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java index 6e5af59c2aeab..93bdd674180bf 100644 --- a/server/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/allocation/FilteringAllocationIT.java @@ -19,10 +19,9 @@ package org.elasticsearch.cluster.allocation; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -41,14 +40,13 @@ import java.util.List; import java.util.Set; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; @ClusterScope(scope= Scope.TEST, numDataNodes =0) public class FilteringAllocationIT extends ESIntegTestCase { - private final Logger logger = LogManager.getLogger(FilteringAllocationIT.class); - - public void testDecommissionNodeNoReplicas() throws Exception { + public void testDecommissionNodeNoReplicas() { logger.info("--> starting 2 nodes"); List nodesIds = internalCluster().startNodes(2); final String node_0 = nodesIds.get(0); @@ -56,10 +54,10 @@ public void testDecommissionNodeNoReplicas() throws Exception { assertThat(cluster().size(), equalTo(2)); logger.info("--> creating an index with no replicas"); - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder().put("index.number_of_replicas", 0)) - .execute().actionGet(); - ensureGreen(); + createIndex("test", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .build()); + ensureGreen("test"); logger.info("--> index some data"); for (int i = 0; i < 100; i++) { client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", "value" + i).execute().actionGet(); @@ -68,11 +66,17 @@ public void testDecommissionNodeNoReplicas() throws Exception { assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet() .getHits().getTotalHits().value, equalTo(100L)); + final boolean closed = randomBoolean(); + if (closed) { + assertAcked(client().admin().indices().prepareClose("test")); + ensureGreen("test"); + } + logger.info("--> decommission the second node"); client().admin().cluster().prepareUpdateSettings() .setTransientSettings(Settings.builder().put("cluster.routing.allocation.exclude._name", node_1)) .execute().actionGet(); - waitForRelocation(); + ensureGreen("test"); logger.info("--> verify all are allocated on node1 now"); ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); @@ -84,12 +88,16 @@ public void testDecommissionNodeNoReplicas() throws Exception { } } + if (closed) { + assertAcked(client().admin().indices().prepareOpen("test")); + } + client().admin().indices().prepareRefresh().execute().actionGet(); assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()) .execute().actionGet().getHits().getTotalHits().value, equalTo(100L)); } - public void testDisablingAllocationFiltering() throws Exception { + public void testDisablingAllocationFiltering() { logger.info("--> starting 2 nodes"); List nodesIds = internalCluster().startNodes(2); final String node_0 = nodesIds.get(0); @@ -97,11 +105,11 @@ public void testDisablingAllocationFiltering() throws Exception { assertThat(cluster().size(), equalTo(2)); logger.info("--> creating an index with no replicas"); - client().admin().indices().prepareCreate("test") - .setSettings(Settings.builder().put("index.number_of_shards", 2).put("index.number_of_replicas", 0)) - .execute().actionGet(); - - ensureGreen(); + createIndex("test", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) + .build()); + ensureGreen("test"); logger.info("--> index some data"); for (int i = 0; i < 100; i++) { @@ -110,6 +118,13 @@ public void testDisablingAllocationFiltering() throws Exception { client().admin().indices().prepareRefresh().execute().actionGet(); assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()) .execute().actionGet().getHits().getTotalHits().value, equalTo(100L)); + + final boolean closed = randomBoolean(); + if (closed) { + assertAcked(client().admin().indices().prepareClose("test")); + ensureGreen("test"); + } + ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); IndexRoutingTable indexRoutingTable = clusterState.routingTable().index("test"); int numShardsOnNode1 = 0; @@ -133,7 +148,7 @@ public void testDisablingAllocationFiltering() throws Exception { .setSettings(Settings.builder().put("index.routing.allocation.exclude._name", node_0)) .execute().actionGet(); client().admin().cluster().prepareReroute().get(); - ensureGreen(); + ensureGreen("test"); logger.info("--> verify all shards are allocated on node_1 now"); clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); @@ -149,7 +164,7 @@ public void testDisablingAllocationFiltering() throws Exception { .setSettings(Settings.builder().put("index.routing.allocation.exclude._name", "")) .execute().actionGet(); client().admin().cluster().prepareReroute().get(); - ensureGreen(); + ensureGreen("test"); logger.info("--> verify that there are shards allocated on both nodes now"); clusterState = client().admin().cluster().prepareState().execute().actionGet().getState(); @@ -166,7 +181,7 @@ public void testInvalidIPFilterClusterSettings() { assertEquals("invalid IP address [192.168.1.1.] for [" + filterSetting.getKey() + ipKey + "]", e.getMessage()); } - public void testTransientSettingsStillApplied() throws Exception { + public void testTransientSettingsStillApplied() { List nodes = internalCluster().startNodes(6); Set excludeNodes = new HashSet<>(nodes.subList(0, 3)); Set includeNodes = new HashSet<>(nodes.subList(3, 6)); @@ -177,6 +192,10 @@ public void testTransientSettingsStillApplied() throws Exception { client().admin().indices().prepareCreate("test").get(); ensureGreen("test"); + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test")); + } + Settings exclude = Settings.builder().put("cluster.routing.allocation.exclude._name", Strings.collectionToCommaDelimitedString(excludeNodes)).build(); diff --git a/server/src/test/java/org/elasticsearch/cluster/allocation/SimpleAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/allocation/SimpleAllocationIT.java index f9c0691576f2b..8f6473b0e359d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/allocation/SimpleAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/allocation/SimpleAllocationIT.java @@ -39,13 +39,12 @@ protected int numberOfReplicas() { return 1; } - /** - * Test for - * https://groups.google.com/d/msg/elasticsearch/y-SY_HyoB-8/EZdfNt9VO44J - */ public void testSaneAllocation() { assertAcked(prepareCreate("test", 3)); - ensureGreen(); + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test")); + } + ensureGreen("test"); ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0)); @@ -56,7 +55,7 @@ public void testSaneAllocation() { } client().admin().indices().prepareUpdateSettings("test") .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, 0)).execute().actionGet(); - ensureGreen(); + ensureGreen("test"); state = client().admin().cluster().prepareState().execute().actionGet().getState(); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0)); @@ -68,11 +67,14 @@ public void testSaneAllocation() { // create another index assertAcked(prepareCreate("test2", 3)); - ensureGreen(); + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareClose("test2")); + } + ensureGreen("test2"); client().admin().indices().prepareUpdateSettings("test") .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, 1)).execute().actionGet(); - ensureGreen(); + ensureGreen("test"); state = client().admin().cluster().prepareState().execute().actionGet().getState(); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0)); From 4db7fd941a447a89bbdade920c599850a86bf5d1 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 25 Feb 2019 13:04:09 +0100 Subject: [PATCH 14/20] Adapt the Recovery API for closed indices (#38421) This commit adapts the Recovery API to make it work with shards of replicated closed indices. Relates #33888 --- .../test/cat.recovery/10_basic.yml | 57 +++++++++++++++++++ .../test/indices.recovery/10_basic.yml | 50 ++++++++++++++++ .../indices/recovery/RecoveryRequest.java | 3 +- .../recovery/TransportRecoveryAction.java | 4 +- .../support/broadcast/BroadcastRequest.java | 5 ++ .../indices/recovery/IndexRecoveryIT.java | 56 ++++++++++++------ 6 files changed, 156 insertions(+), 19 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml index 69ceccc1ef3bf..83194db1fa5a3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml @@ -75,3 +75,60 @@ \n )+ $/ + +--- +"Test cat recovery output for closed index": + - skip: + version: " - 7.99.99" + reason: closed indices are replicated starting version 8.0.0 + + - do: + indices.create: + index: index2 + body: + settings: + index: + number_of_replicas: 0 + + - do: + indices.close: + index: index2 + - is_true: acknowledged + + - do: + cluster.health: + index: index2 + wait_for_status: green + + - do: + cat.recovery: + index: index2 + h: i,s,t,ty,st,shost,thost,rep,snap,f,fr,fp,tf,b,br,bp,tb,to,tor,top + + - match: + $body: | + /^ + ( + index2 \s+ + \d \s+ # shard + (?:\d+ms|\d+(?:\.\d+)?s) \s+ # time in ms or seconds + existing_store \s+ # source type (always existing_store for closed indices) + done \s+ # stage + [-\w./]+ \s+ # source_host + [-\w./]+ \s+ # target_host + [-\w./]+ \s+ # repository + [-\w./]+ \s+ # snapshot + \d+ \s+ # files + \d+ \s+ # files_recovered + \d+\.\d+% \s+ # files_percent + \d+ \s+ # files_total + \d+ \s+ # bytes + \d+ \s+ # bytes_recovered + \d+\.\d+% \s+ # bytes_percent + \d+ \s+ # bytes_total + 0 \s+ # translog_ops (always 0 for closed indices) + 0 \s+ # translog_ops_recovered (always 0 for closed indices) + 100\.0% # translog_ops_percent (always 100.0% for closed indices) + \n + )+ + $/ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml index fd8937a23cdee..07fe657e77ff3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.recovery/10_basic.yml @@ -40,6 +40,56 @@ - gte: { test_1.shards.0.verify_index.check_index_time_in_millis: 0 } - gte: { test_1.shards.0.verify_index.total_time_in_millis: 0 } --- +"Indices recovery test for closed index": + - skip: + version: " - 7.99.99" + reason: closed indices are replicated starting version 8.0.0 + + - do: + indices.create: + index: test_2 + body: + settings: + index: + number_of_replicas: 0 + + - do: + indices.close: + index: test_2 + - is_true: acknowledged + + - do: + cluster.health: + index: test_2 + wait_for_status: green + + - do: + indices.recovery: + index: [test_2] + human: true + + - match: { test_2.shards.0.type: "EXISTING_STORE" } + - match: { test_2.shards.0.stage: "DONE" } + - match: { test_2.shards.0.primary: true } + - match: { test_2.shards.0.start_time: /^2\d\d\d-.+/ } + - match: { test_2.shards.0.target.ip: /^\d+\.\d+\.\d+\.\d+$/ } + - gte: { test_2.shards.0.index.files.total: 0 } + - gte: { test_2.shards.0.index.files.reused: 0 } + - gte: { test_2.shards.0.index.files.recovered: 0 } + - match: { test_2.shards.0.index.files.percent: /^\d+\.\d\%$/ } + - gte: { test_2.shards.0.index.size.total_in_bytes: 0 } + - gte: { test_2.shards.0.index.size.reused_in_bytes: 0 } + - gte: { test_2.shards.0.index.size.recovered_in_bytes: 0 } + - match: { test_2.shards.0.index.size.percent: /^\d+\.\d\%$/ } + - gte: { test_2.shards.0.index.source_throttle_time_in_millis: 0 } + - gte: { test_2.shards.0.index.target_throttle_time_in_millis: 0 } + - gte: { test_2.shards.0.translog.recovered: 0 } + - gte: { test_2.shards.0.translog.total: 0 } + - gte: { test_2.shards.0.translog.total_on_start: 0 } + - gte: { test_2.shards.0.translog.total_time_in_millis: 0 } + - gte: { test_2.shards.0.verify_index.check_index_time_in_millis: 0 } + - gte: { test_2.shards.0.verify_index.total_time_in_millis: 0 } +--- "Indices recovery test index name not matching": - do: diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryRequest.java index 8878713765ba0..78d2969c2fde4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/RecoveryRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.admin.indices.recovery; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.broadcast.BroadcastRequest; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -47,7 +48,7 @@ public RecoveryRequest() { * @param indices Comma-separated list of indices about which to gather recovery information */ public RecoveryRequest(String... indices) { - super(indices); + super(indices, IndicesOptions.STRICT_EXPAND_OPEN_CLOSED); } /** diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/TransportRecoveryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/TransportRecoveryAction.java index f7356bd242d06..0ff31f42b9295 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/TransportRecoveryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/recovery/TransportRecoveryAction.java @@ -112,11 +112,11 @@ protected ShardsIterator shards(ClusterState state, RecoveryRequest request, Str @Override protected ClusterBlockException checkGlobalBlock(ClusterState state, RecoveryRequest request) { - return state.blocks().globalBlockedException(ClusterBlockLevel.READ); + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); } @Override protected ClusterBlockException checkRequestBlock(ClusterState state, RecoveryRequest request, String[] concreteIndices) { - return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices); + return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastRequest.java b/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastRequest.java index a04d2edc8dc63..6cf42e7ad3f14 100644 --- a/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastRequest.java +++ b/server/src/main/java/org/elasticsearch/action/support/broadcast/BroadcastRequest.java @@ -40,6 +40,11 @@ protected BroadcastRequest(String[] indices) { this.indices = indices; } + protected BroadcastRequest(String[] indices, IndicesOptions indicesOptions) { + this.indices = indices; + this.indicesOptions = indicesOptions; + } + @Override public String[] indices() { return indices; diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index ea3e933a88314..82d6c38becaec 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -54,6 +54,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -209,24 +210,34 @@ public void testGatewayRecoveryTestActiveOnly() throws Exception { } public void testReplicaRecovery() throws Exception { - logger.info("--> start node A"); - String nodeA = internalCluster().startNode(); + final String nodeA = internalCluster().startNode(); + createIndex(INDEX_NAME, Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, SHARD_COUNT) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, REPLICA_COUNT) + .build()); + ensureGreen(INDEX_NAME); + + final int numOfDocs = scaledRandomIntBetween(0, 200); + try (BackgroundIndexer indexer = new BackgroundIndexer(INDEX_NAME, "_doc", client(), numOfDocs)) { + waitForDocs(numOfDocs, indexer); + } - logger.info("--> create index on node: {}", nodeA); - createAndPopulateIndex(INDEX_NAME, 1, SHARD_COUNT, REPLICA_COUNT); + refresh(INDEX_NAME); + assertHitCount(client().prepareSearch(INDEX_NAME).setSize(0).get(), numOfDocs); - logger.info("--> start node B"); - String nodeB = internalCluster().startNode(); - ensureGreen(); + final boolean closedIndex = randomBoolean(); + if (closedIndex) { + assertAcked(client().admin().indices().prepareClose(INDEX_NAME)); + ensureGreen(INDEX_NAME); + } // force a shard recovery from nodeA to nodeB - logger.info("--> bump replica count"); - client().admin().indices().prepareUpdateSettings(INDEX_NAME) - .setSettings(Settings.builder().put("number_of_replicas", 1)).execute().actionGet(); - ensureGreen(); + final String nodeB = internalCluster().startNode(); + assertAcked(client().admin().indices().prepareUpdateSettings(INDEX_NAME) + .setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1))); + ensureGreen(INDEX_NAME); - logger.info("--> request recoveries"); - RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); + final RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); // we should now have two total shards, one primary and one replica List recoveryStates = response.shardRecoveryStates().get(INDEX_NAME); @@ -238,14 +249,27 @@ public void testReplicaRecovery() throws Exception { assertThat(nodeBResponses.size(), equalTo(1)); // validate node A recovery - RecoveryState nodeARecoveryState = nodeAResponses.get(0); - assertRecoveryState(nodeARecoveryState, 0, RecoverySource.EmptyStoreRecoverySource.INSTANCE, true, Stage.DONE, null, nodeA); + final RecoveryState nodeARecoveryState = nodeAResponses.get(0); + final RecoverySource expectedRecoverySource; + if (closedIndex == false) { + expectedRecoverySource = RecoverySource.EmptyStoreRecoverySource.INSTANCE; + } else { + expectedRecoverySource = RecoverySource.ExistingStoreRecoverySource.INSTANCE; + } + assertRecoveryState(nodeARecoveryState, 0, expectedRecoverySource, true, Stage.DONE, null, nodeA); validateIndexRecoveryState(nodeARecoveryState.getIndex()); // validate node B recovery - RecoveryState nodeBRecoveryState = nodeBResponses.get(0); + final RecoveryState nodeBRecoveryState = nodeBResponses.get(0); assertRecoveryState(nodeBRecoveryState, 0, PeerRecoverySource.INSTANCE, false, Stage.DONE, nodeA, nodeB); validateIndexRecoveryState(nodeBRecoveryState.getIndex()); + + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeA)); + + if (closedIndex) { + assertAcked(client().admin().indices().prepareOpen(INDEX_NAME)); + } + assertHitCount(client().prepareSearch(INDEX_NAME).setSize(0).get(), numOfDocs); } @TestLogging( From 71f5c3495dd9eb1d5d552799f44ea505db68eb36 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 26 Feb 2019 09:29:50 +0100 Subject: [PATCH 15/20] Recover closed indices after a full cluster restart (#39249) Closing an index is a process that can be broken down into several steps: 1. first, the state of the cluster is updated to add a write block on the index to be closed 2. then, a transport replication action is executed on all shards of the index. This action checks that the maximum sequence number and the global checkpoint have identical values, indicating that all in flight writing operations have been completed on the shard. 3. finally, and if the previous steps were successful, the cluster state is updated again to change the state of the index from `OPEN`to `CLOSE`. During the last step, the master node retrieves the minimum node version among all the nodes that compose the cluster: * If a node is in pre 8.0 version, the index is closed and the index routing table is removed from the cluster state. This is the "old" way of closing indices and closed indices with no routing table are not replicated. * If all nodes are in version 8.0 or higher, the index is closed and its routing table is reinitialized in cluster state. This is the new way of closing indices and such closed indices will be replicated in the cluster. But routing tables are not persisted in the cluster state, so after a full cluster restart there is no way to make the distinction between an index closed in 7.x and an index closed and replicated on 8.0. This commit introduces a new private index settings named `index.verified_before_close` that is added to closed indices that are replicated at closing time. This setting serves as a marker to indicate that the index has been closed using the new Close Index API on a cluster that supports replication of closed indices. This way, after a full cluster restart, the Gateway service can automatically recovers those closed indices as if they were opened indices. Closed indices that don't have this setting (because they were closed on a pre-8.0 cluster, or a cluster in mixed version) won't be recovered and will need to be reopened and closed again on a 8.0 cluster. Note that reopening the index removes the private setting. Relates to #33888 --- .../upgrades/FullClusterRestartIT.java | 96 ++++++++++++ .../elasticsearch/upgrades/RecoveryIT.java | 147 ++++++++++++++++++ .../metadata/MetaDataIndexStateService.java | 30 +++- .../cluster/routing/RoutingTable.java | 8 +- .../common/settings/IndexScopedSettings.java | 2 + .../cluster/allocation/ClusterRerouteIT.java | 23 ++- .../MetaDataIndexStateServiceTests.java | 57 ++++++- .../cluster/routing/RoutingTableTests.java | 32 ++++ .../gateway/ClusterStateUpdatersTests.java | 36 ++++- .../gateway/GatewayIndexStateIT.java | 3 +- .../indices/state/CloseIndexIT.java | 10 +- .../xpack/ccr/IndexFollowingIT.java | 2 +- .../index/engine/FrozenIndexTests.java | 2 +- 13 files changed, 418 insertions(+), 30 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 b34f677e1c15b..36cb6bcadd73b 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 @@ -26,6 +26,7 @@ import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Strings; @@ -41,6 +42,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Base64; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -59,8 +61,11 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; /** * Tests to run before and after a full cluster restart. This is run twice, @@ -951,6 +956,97 @@ public void testSoftDeletes() throws Exception { } } + /** + * This test creates an index in the old cluster and then closes it. When the cluster is fully restarted in a newer version, + * it verifies that the index exists and is replicated if the old version supports replication. + */ + public void testClosedIndices() throws Exception { + if (isRunningAgainstOldCluster()) { + createIndex(index, Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .build()); + ensureGreen(index); + + int numDocs = 0; + if (randomBoolean()) { + numDocs = between(1, 100); + for (int i = 0; i < numDocs; i++) { + final Request request = new Request("POST", "/" + index + "/_doc/" + i); + request.setJsonEntity(Strings.toString(JsonXContent.contentBuilder().startObject().field("field", "v1").endObject())); + assertOK(client().performRequest(request)); + if (rarely()) { + refresh(); + } + } + refresh(); + } + + assertTotalHits(numDocs, entityAsMap(client().performRequest(new Request("GET", "/" + index + "/_search")))); + saveInfoDocument(index + "_doc_count", Integer.toString(numDocs)); + closeIndex(index); + } + + if (getOldClusterVersion().onOrAfter(Version.V_8_0_0)) { + ensureGreenLongWait(index); + assertClosedIndex(index, true); + } else { + assertClosedIndex(index, false); + } + + if (isRunningAgainstOldCluster() == false) { + openIndex(index); + ensureGreen(index); + + final int expectedNumDocs = Integer.parseInt(loadInfoDocument(index + "_doc_count")); + assertTotalHits(expectedNumDocs, entityAsMap(client().performRequest(new Request("GET", "/" + index + "/_search")))); + } + } + + /** + * Asserts that an index is closed in the cluster state. If `checkRoutingTable` is true, it also asserts + * that the index has started shards. + */ + @SuppressWarnings("unchecked") + private void assertClosedIndex(final String index, final boolean checkRoutingTable) throws IOException { + final Map state = entityAsMap(client().performRequest(new Request("GET", "/_cluster/state"))); + + final Map metadata = (Map) XContentMapValues.extractValue("metadata.indices." + index, state); + assertThat(metadata, notNullValue()); + assertThat(metadata.get("state"), equalTo("close")); + + final Map blocks = (Map) XContentMapValues.extractValue("blocks.indices." + index, state); + assertThat(blocks, notNullValue()); + assertThat(blocks.containsKey(String.valueOf(MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID)), is(true)); + + final Map settings = (Map) XContentMapValues.extractValue("settings", metadata); + assertThat(settings, notNullValue()); + + final Map routingTable = (Map) XContentMapValues.extractValue("routing_table.indices." + index, state); + if (checkRoutingTable) { + assertThat(routingTable, notNullValue()); + assertThat(Booleans.parseBoolean((String) XContentMapValues.extractValue("index.verified_before_close", settings)), is(true)); + final String numberOfShards = (String) XContentMapValues.extractValue("index.number_of_shards", settings); + assertThat(numberOfShards, notNullValue()); + final int nbShards = Integer.parseInt(numberOfShards); + assertThat(nbShards, greaterThanOrEqualTo(1)); + + for (int i = 0; i < nbShards; i++) { + final Collection> shards = + (Collection>) XContentMapValues.extractValue("shards." + i, routingTable); + assertThat(shards, notNullValue()); + assertThat(shards.size(), equalTo(2)); + for (Map shard : shards) { + assertThat(XContentMapValues.extractValue("shard", shard), equalTo(i)); + assertThat(XContentMapValues.extractValue("state", shard), equalTo("STARTED")); + assertThat(XContentMapValues.extractValue("index", shard), equalTo(index)); + } + } + } else { + assertThat(routingTable, nullValue()); + assertThat(XContentMapValues.extractValue("index.verified_before_close", settings), nullValue()); + } + } + private void checkSnapshot(final String snapshotName, final int count, final Version tookOnVersion) throws IOException { // Check the snapshot metadata, especially the version Request listSnapshotRequest = new Request("GET", "/_snapshot/repo/" + snapshotName); diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java index 295aee8b869ff..402c0c4859b77 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java @@ -24,15 +24,20 @@ import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; +import org.elasticsearch.common.Booleans; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.rest.action.document.RestIndexAction; import org.elasticsearch.test.rest.yaml.ObjectPath; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Future; import java.util.function.Predicate; @@ -43,7 +48,9 @@ import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; /** * In depth testing of the recovery mechanism during a rolling restart. @@ -310,4 +317,144 @@ public void testRecoveryWithSoftDeletes() throws Exception { } ensureGreen(index); } + + /** + * This test creates an index in the non upgraded cluster and closes it. It then checks that the index + * is effectively closed and potentially replicated (if the version the index was created on supports + * the replication of closed indices) during the rolling upgrade. + */ + public void testRecoveryClosedIndex() throws Exception { + final String indexName = "closed_index_created_on_old"; + if (CLUSTER_TYPE == ClusterType.OLD) { + createIndex(indexName, Settings.builder() + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1) + // if the node with the replica is the first to be restarted, while a replica is still recovering + // then delayed allocation will kick in. When the node comes back, the master will search for a copy + // but the recovering copy will be seen as invalid and the cluster health won't return to GREEN + // before timing out + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") + .put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0") // fail faster + .build()); + ensureGreen(indexName); + closeIndex(indexName); + } + + final Version indexVersionCreated = indexVersionCreated(indexName); + if (indexVersionCreated.onOrAfter(Version.V_8_0_0)) { + // index was created on a version that supports the replication of closed indices, + // so we expect the index to be closed and replicated + ensureGreen(indexName); + assertClosedIndex(indexName, true); + } else { + assertClosedIndex(indexName, false); + } + } + + /** + * This test creates and closes a new index at every stage of the rolling upgrade. It then checks that the index + * is effectively closed and potentially replicated if the cluster supports replication of closed indices at the + * time the index was closed. + */ + public void testCloseIndexDuringRollingUpgrade() throws Exception { + final Version minimumNodeVersion = minimumNodeVersion(); + final String indexName = + String.join("_", "index", CLUSTER_TYPE.toString(), Integer.toString(minimumNodeVersion.id)).toLowerCase(Locale.ROOT); + + if (indexExists(indexName) == false) { + createIndex(indexName, Settings.builder() + .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) + .build()); + ensureGreen(indexName); + closeIndex(indexName); + } + + if (minimumNodeVersion.onOrAfter(Version.V_8_0_0)) { + // index is created on a version that supports the replication of closed indices, + // so we expect the index to be closed and replicated + ensureGreen(indexName); + assertClosedIndex(indexName, true); + } else { + assertClosedIndex(indexName, false); + } + } + + /** + * Returns the version in which the given index has been created + */ + private static Version indexVersionCreated(final String indexName) throws IOException { + final Request request = new Request("GET", "/" + indexName + "/_settings"); + final String versionCreatedSetting = indexName + ".settings.index.version.created"; + request.addParameter("filter_path", versionCreatedSetting); + + final Response response = client().performRequest(request); + return Version.fromId(Integer.parseInt(ObjectPath.createFromResponse(response).evaluate(versionCreatedSetting))); + } + + /** + * Returns the minimum node version among all nodes of the cluster + */ + private static Version minimumNodeVersion() throws IOException { + final Request request = new Request("GET", "_nodes"); + request.addParameter("filter_path", "nodes.*.version"); + + final Response response = client().performRequest(request); + final Map nodes = ObjectPath.createFromResponse(response).evaluate("nodes"); + + Version minVersion = null; + for (Map.Entry node : nodes.entrySet()) { + @SuppressWarnings("unchecked") + Version nodeVersion = Version.fromString((String) ((Map) node.getValue()).get("version")); + if (minVersion == null || minVersion.after(nodeVersion)) { + minVersion = nodeVersion; + } + } + assertNotNull(minVersion); + return minVersion; + } + + /** + * Asserts that an index is closed in the cluster state. If `checkRoutingTable` is true, it also asserts + * that the index has started shards. + */ + @SuppressWarnings("unchecked") + private void assertClosedIndex(final String index, final boolean checkRoutingTable) throws IOException { + final Map state = entityAsMap(client().performRequest(new Request("GET", "/_cluster/state"))); + + final Map metadata = (Map) XContentMapValues.extractValue("metadata.indices." + index, state); + assertThat(metadata, notNullValue()); + assertThat(metadata.get("state"), equalTo("close")); + + final Map blocks = (Map) XContentMapValues.extractValue("blocks.indices." + index, state); + assertThat(blocks, notNullValue()); + assertThat(blocks.containsKey(String.valueOf(MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID)), is(true)); + + final Map settings = (Map) XContentMapValues.extractValue("settings", metadata); + assertThat(settings, notNullValue()); + + final int numberOfShards = Integer.parseInt((String) XContentMapValues.extractValue("index.number_of_shards", settings)); + final int numberOfReplicas = Integer.parseInt((String) XContentMapValues.extractValue("index.number_of_replicas", settings)); + + final Map routingTable = (Map) XContentMapValues.extractValue("routing_table.indices." + index, state); + if (checkRoutingTable) { + assertThat(routingTable, notNullValue()); + assertThat(Booleans.parseBoolean((String) XContentMapValues.extractValue("index.verified_before_close", settings)), is(true)); + + for (int i = 0; i < numberOfShards; i++) { + final Collection> shards = + (Collection>) XContentMapValues.extractValue("shards." + i, routingTable); + assertThat(shards, notNullValue()); + assertThat(shards.size(), equalTo(numberOfReplicas + 1)); + for (Map shard : shards) { + assertThat(XContentMapValues.extractValue("shard", shard), equalTo(i)); + assertThat(XContentMapValues.extractValue("state", shard), equalTo("STARTED")); + assertThat(XContentMapValues.extractValue("index", shard), equalTo(index)); + } + } + } else { + assertThat(routingTable, nullValue()); + assertThat(XContentMapValues.extractValue("index.verified_before_close", settings), nullValue()); + } + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index ca58b49e51860..4d81bf6e9c557 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -52,6 +52,8 @@ import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; @@ -90,6 +92,8 @@ public class MetaDataIndexStateService { public static final int INDEX_CLOSED_BLOCK_ID = 4; public static final ClusterBlock INDEX_CLOSED_BLOCK = new ClusterBlock(4, "index closed", false, false, false, RestStatus.FORBIDDEN, ClusterBlockLevel.READ_WRITE); + public static final Setting VERIFIED_BEFORE_CLOSE_SETTING = + Setting.boolSetting("index.verified_before_close", false, Setting.Property.IndexScope, Setting.Property.PrivateIndex); private final ClusterService clusterService; private final AllocationService allocationService; @@ -402,15 +406,22 @@ static ClusterState closeRoutingTable(final ClusterState currentState, continue; } - logger.debug("closing index {} succeeded", index); - metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID); blocks.addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + final IndexMetaData.Builder updatedMetaData = IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE); if (removeRoutingTable) { + metadata.put(updatedMetaData); routingTable.remove(index.getName()); } else { + metadata.put(updatedMetaData + .settingsVersion(indexMetaData.getSettingsVersion() + 1) + .settings(Settings.builder() + .put(indexMetaData.getSettings()) + .put(VERIFIED_BEFORE_CLOSE_SETTING.getKey(), true))); routingTable.addAsFromOpenToClose(metadata.getSafe(index)); } + + logger.debug("closing index {} succeeded", index); closedIndices.add(index.getName()); } catch (final IndexNotFoundException e) { logger.debug("index {} has been deleted since it was blocked before closing, ignoring", index); @@ -490,7 +501,15 @@ ClusterState openIndices(final Index[] indices, final ClusterState currentState) for (IndexMetaData indexMetaData : indicesToOpen) { final Index index = indexMetaData.getIndex(); if (indexMetaData.getState() != IndexMetaData.State.OPEN) { - IndexMetaData updatedIndexMetaData = IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.OPEN).build(); + final Settings.Builder updatedSettings = Settings.builder().put(indexMetaData.getSettings()); + updatedSettings.remove(VERIFIED_BEFORE_CLOSE_SETTING.getKey()); + + IndexMetaData updatedIndexMetaData = IndexMetaData.builder(indexMetaData) + .state(IndexMetaData.State.OPEN) + .settingsVersion(indexMetaData.getSettingsVersion() + 1) + .settings(updatedSettings) + .build(); + // The index might be closed because we couldn't import it due to old incompatible version // We need to check that this index can be upgraded to the current version updatedIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(updatedIndexMetaData, minIndexCompatibilityVersion); @@ -554,4 +573,9 @@ public static ClusterBlock createIndexClosingBlock() { EnumSet.of(ClusterBlockLevel.WRITE)); } + public static boolean isIndexVerifiedBeforeClosed(final IndexMetaData indexMetaData) { + return indexMetaData.getState() == IndexMetaData.State.CLOSE + && VERIFIED_BEFORE_CLOSE_SETTING.exists(indexMetaData.getSettings()) + && VERIFIED_BEFORE_CLOSE_SETTING.get(indexMetaData.getSettings()); + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 32ec3e7ce5ba8..3a49577563929 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -47,6 +47,8 @@ import java.util.Map; import java.util.function.Predicate; +import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.isIndexVerifiedBeforeClosed; + /** * Represents a global cluster-wide routing table for all indices including the * version of the current routing state. @@ -499,9 +501,9 @@ public Builder addAsNew(IndexMetaData indexMetaData) { } public Builder addAsRecovery(IndexMetaData indexMetaData) { - if (indexMetaData.getState() == IndexMetaData.State.OPEN) { + if (indexMetaData.getState() == IndexMetaData.State.OPEN || isIndexVerifiedBeforeClosed(indexMetaData)) { IndexRoutingTable.Builder indexRoutingBuilder = new IndexRoutingTable.Builder(indexMetaData.getIndex()) - .initializeAsRecovery(indexMetaData); + .initializeAsRecovery(indexMetaData); add(indexRoutingBuilder); } return this; @@ -526,7 +528,7 @@ public Builder addAsFromCloseToOpen(IndexMetaData indexMetaData) { } public Builder addAsFromOpenToClose(IndexMetaData indexMetaData) { - assert indexMetaData.getState() == IndexMetaData.State.CLOSE; + assert isIndexVerifiedBeforeClosed(indexMetaData); IndexRoutingTable.Builder indexRoutingBuilder = new IndexRoutingTable.Builder(indexMetaData.getIndex()) .initializeAsFromOpenToClose(indexMetaData); return add(indexRoutingBuilder); diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 9d936a28846e4..f1e42d2413815 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.settings; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; @@ -161,6 +162,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { EngineConfig.INDEX_CODEC_SETTING, IndexMetaData.SETTING_WAIT_FOR_ACTIVE_SHARDS, IndexSettings.DEFAULT_PIPELINE, + MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING, // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { diff --git a/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java b/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java index 95cb09df0f68c..d629804b02d58 100644 --- a/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java @@ -237,6 +237,11 @@ private void rerouteWithAllocateLocalGateway(Settings commonSettings) throws Exc .setSettings(Settings.builder().put("index.number_of_shards", 1)) .execute().actionGet(); + final boolean closed = randomBoolean(); + if (closed) { + client().admin().indices().prepareClose("test").get(); + } + ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); assertThat(state.getRoutingNodes().unassigned().size(), equalTo(2)); @@ -249,8 +254,11 @@ private void rerouteWithAllocateLocalGateway(Settings commonSettings) throws Exc assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.INITIALIZING)); - healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID) - .setWaitForYellowStatus().execute().actionGet(); + healthResponse = client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForEvents(Priority.LANGUID) + .setWaitForYellowStatus() + .execute().actionGet(); assertThat(healthResponse.isTimedOut(), equalTo(false)); logger.info("--> get the state, verify shard 1 primary allocated"); @@ -259,8 +267,10 @@ private void rerouteWithAllocateLocalGateway(Settings commonSettings) throws Exc assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.STARTED)); - client().prepareIndex("test", "type", "1").setSource("field", "value") - .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); + if (closed == false) { + client().prepareIndex("test", "type", "1").setSource("field", "value") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); + } final Index index = resolveIndex("test"); logger.info("--> closing all nodes"); @@ -278,7 +288,10 @@ private void rerouteWithAllocateLocalGateway(Settings commonSettings) throws Exc // wait a bit for the cluster to realize that the shard is not there... // TODO can we get around this? the cluster is RED, so what do we wait for? client().admin().cluster().prepareReroute().get(); - assertThat(client().admin().cluster().prepareHealth().setWaitForNodes("2").execute().actionGet().getStatus(), + assertThat(client().admin().cluster().prepareHealth() + .setIndices("test") + .setWaitForNodes("2") + .execute().actionGet().getStatus(), equalTo(ClusterHealthStatus.RED)); logger.info("--> explicitly allocate primary"); state = client().admin().cluster().prepareReroute() diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index df07982f445ba..6ba85cd22a36e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -152,7 +152,10 @@ public void testCloseRoutingTableRemovesRoutingTable() { } for (Index blockedIndex : blockedIndices.keySet()) { if (results.get(blockedIndex).isAcknowledged()) { - assertThat(state.metaData().index(blockedIndex).getState(), is(IndexMetaData.State.CLOSE)); + IndexMetaData indexMetaData = state.metaData().index(blockedIndex); + assertThat(indexMetaData.getState(), is(IndexMetaData.State.CLOSE)); + Settings indexSettings = indexMetaData.getSettings(); + assertThat(indexSettings.hasValue(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()), is(false)); assertThat(state.blocks().hasIndexBlock(blockedIndex.getName(), MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); assertThat("Index must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", state.blocks().indices().getOrDefault(blockedIndex.getName(), emptySet()).stream() @@ -191,7 +194,6 @@ public void testAddIndexClosedBlocks() { ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); assertSame(state, updatedState); assertTrue(blockedIndices.isEmpty()); - } { final Map blockedIndices = new HashMap<>(); @@ -302,6 +304,32 @@ public void testValidateShardLimit() { currentShards + "]/[" + maxShards + "] maximum shards open;", exception.getMessage()); } + public void testIsIndexVerifiedBeforeClosed() { + final ClusterState initialState = ClusterState.builder(new ClusterName("testIsIndexMetaDataClosed")).build(); + { + String indexName = "open"; + ClusterState state = addOpenedIndex(indexName, randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + assertFalse(MetaDataIndexStateService.isIndexVerifiedBeforeClosed(state.getMetaData().index(indexName))); + } + { + String indexName = "closed"; + ClusterState state = addClosedIndex(indexName, randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + assertTrue(MetaDataIndexStateService.isIndexVerifiedBeforeClosed(state.getMetaData().index(indexName))); + } + { + String indexName = "closed-no-setting"; + IndexMetaData indexMetaData = IndexMetaData.builder(indexName) + .state(IndexMetaData.State.CLOSE) + .creationDate(randomNonNegativeLong()) + .settings(Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 3)) + .put(SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 3))) + .build(); + assertFalse(MetaDataIndexStateService.isIndexVerifiedBeforeClosed(indexMetaData)); + } + } + public static ClusterState createClusterForShardLimitTest(int nodesInCluster, int openIndexShards, int openIndexReplicas, int closedIndexShards, int closedIndexReplicas, Settings clusterSettings) { ImmutableOpenMap.Builder dataNodes = ImmutableOpenMap.builder(); @@ -374,13 +402,18 @@ private static ClusterState addIndex(final ClusterState currentState, final int numReplicas, final IndexMetaData.State state, @Nullable final ClusterBlock block) { + + final Settings.Builder settings = Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_NUMBER_OF_SHARDS, numShards) + .put(SETTING_NUMBER_OF_REPLICAS, numReplicas); + if (state == IndexMetaData.State.CLOSE) { + settings.put(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey(), true); + } final IndexMetaData indexMetaData = IndexMetaData.builder(index) .state(state) .creationDate(randomNonNegativeLong()) - .settings(Settings.builder() - .put(SETTING_VERSION_CREATED, Version.CURRENT) - .put(SETTING_NUMBER_OF_SHARDS, numShards) - .put(SETTING_NUMBER_OF_REPLICAS, numReplicas)) + .settings(settings) .build(); final ClusterState.Builder clusterStateBuilder = ClusterState.builder(currentState); @@ -405,12 +438,20 @@ private static ClusterState addIndex(final ClusterState currentState, } private static void assertIsOpened(final String indexName, final ClusterState clusterState) { - assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.OPEN)); + final IndexMetaData indexMetaData = clusterState.metaData().indices().get(indexName); + assertThat(indexMetaData.getState(), is(IndexMetaData.State.OPEN)); + assertThat(indexMetaData.getSettings().hasValue(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()), is(false)); + assertThat(clusterState.routingTable().index(indexName), notNullValue()); + assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(false)); assertThat(clusterState.routingTable().index(indexName), notNullValue()); } private static void assertIsClosed(final String indexName, final ClusterState clusterState) { - assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.CLOSE)); + final IndexMetaData indexMetaData = clusterState.metaData().indices().get(indexName); + assertThat(indexMetaData.getState(), is(IndexMetaData.State.CLOSE)); + final Settings indexSettings = indexMetaData.getSettings(); + assertThat(indexSettings.hasValue(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()), is(true)); + assertThat(indexSettings.getAsBoolean(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey(), false), is(true)); assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); assertThat("Index " + indexName + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", clusterState.blocks().indices().getOrDefault(indexName, emptySet()).stream() diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java index a6c2fab5c91e4..851fe9c550270 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes.Builder; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -38,6 +39,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; +import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -373,6 +375,36 @@ public void testDistinctNodes() { assertFalse(IndexShardRoutingTable.Builder.distinctNodes(Arrays.asList(routing2, routing4))); } + public void testAddAsRecovery() { + { + final IndexMetaData indexMetaData = createIndexMetaData(TEST_INDEX_1).state(IndexMetaData.State.OPEN).build(); + final RoutingTable routingTable = new RoutingTable.Builder().addAsRecovery(indexMetaData).build(); + assertThat(routingTable.hasIndex(TEST_INDEX_1), is(true)); + assertThat(routingTable.allShards(TEST_INDEX_1).size(), is(this.shardsPerIndex)); + assertThat(routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size(), is(this.shardsPerIndex)); + } + { + final IndexMetaData indexMetaData = createIndexMetaData(TEST_INDEX_1).state(IndexMetaData.State.CLOSE).build(); + final RoutingTable routingTable = new RoutingTable.Builder().addAsRecovery(indexMetaData).build(); + assertThat(routingTable.hasIndex(TEST_INDEX_1), is(false)); + expectThrows(IndexNotFoundException.class, () -> routingTable.allShards(TEST_INDEX_1)); + } + { + final IndexMetaData indexMetaData = createIndexMetaData(TEST_INDEX_1).build(); + final IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(indexMetaData) + .state(IndexMetaData.State.CLOSE) + .settings(Settings.builder() + .put(indexMetaData.getSettings()) + .put(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey(), true) + .build()) + .settingsVersion(indexMetaData.getSettingsVersion() + 1); + final RoutingTable routingTable = new RoutingTable.Builder().addAsRecovery(indexMetaDataBuilder.build()).build(); + assertThat(routingTable.hasIndex(TEST_INDEX_1), is(true)); + assertThat(routingTable.allShards(TEST_INDEX_1).size(), is(this.shardsPerIndex)); + assertThat(routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size(), is(this.shardsPerIndex)); + } + } + /** reverse engineer the in sync aid based on the given indexRoutingTable **/ public static IndexMetaData updateActiveAllocations(IndexRoutingTable indexRoutingTable, IndexMetaData indexMetaData) { IndexMetaData.Builder imdBuilder = IndexMetaData.builder(indexMetaData); diff --git a/server/src/test/java/org/elasticsearch/gateway/ClusterStateUpdatersTests.java b/server/src/test/java/org/elasticsearch/gateway/ClusterStateUpdatersTests.java index cae33db90a6bc..eec02438f0031 100644 --- a/server/src/test/java/org/elasticsearch/gateway/ClusterStateUpdatersTests.java +++ b/server/src/test/java/org/elasticsearch/gateway/ClusterStateUpdatersTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.coordination.CoordinationMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; @@ -241,11 +242,36 @@ public void testUpdateRoutingTable() { .build(); assertFalse(initialState.routingTable().hasIndex(index)); - final ClusterState newState = updateRoutingTable(initialState); - - assertTrue(newState.routingTable().hasIndex(index)); - assertThat(newState.routingTable().version(), is(0L)); - assertThat(newState.routingTable().allShards(index.getName()).size(), is(numOfShards)); + { + final ClusterState newState = updateRoutingTable(initialState); + assertTrue(newState.routingTable().hasIndex(index)); + assertThat(newState.routingTable().version(), is(0L)); + assertThat(newState.routingTable().allShards(index.getName()).size(), is(numOfShards)); + } + { + final ClusterState newState = updateRoutingTable(ClusterState.builder(initialState) + .metaData(MetaData.builder(initialState.metaData()) + .put(IndexMetaData.builder(initialState.metaData().index("test")) + .state(IndexMetaData.State.CLOSE)) + .build()) + .build()); + assertFalse(newState.routingTable().hasIndex(index)); + } + { + final ClusterState newState = updateRoutingTable(ClusterState.builder(initialState) + .metaData(MetaData.builder(initialState.metaData()) + .put(IndexMetaData.builder(initialState.metaData().index("test")) + .state(IndexMetaData.State.CLOSE) + .settings(Settings.builder() + .put(initialState.metaData().index("test").getSettings()) + .put(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey(), true) + .build()) + ).build()) + .build()); + assertTrue(newState.routingTable().hasIndex(index)); + assertThat(newState.routingTable().version(), is(0L)); + assertThat(newState.routingTable().allShards(index.getName()).size(), is(numOfShards)); + } } public void testMixCurrentAndRecoveredState() { diff --git a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index 541a24247473a..5b30b85cb72ec 100644 --- a/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -59,7 +59,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; @ClusterScope(scope = Scope.TEST, numDataNodes = 0) public class GatewayIndexStateIT extends ESIntegTestCase { @@ -166,7 +165,7 @@ public void testSimpleOpenClose() throws Exception { stateResponse = client().admin().cluster().prepareState().execute().actionGet(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); - assertThat(stateResponse.getState().routingTable().index("test"), nullValue()); + assertThat(stateResponse.getState().routingTable().index("test"), notNullValue()); logger.info("--> trying to index into a closed index ..."); try { diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java index ca3f6e694097d..42f29e99982cc 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java @@ -308,7 +308,11 @@ public void testConcurrentClosesAndOpens() throws Exception { static void assertIndexIsClosed(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); for (String index : indices) { - assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.CLOSE)); + final IndexMetaData indexMetaData = clusterState.metaData().indices().get(index); + assertThat(indexMetaData.getState(), is(IndexMetaData.State.CLOSE)); + final Settings indexSettings = indexMetaData.getSettings(); + assertThat(indexSettings.hasValue(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()), is(true)); + assertThat(indexSettings.getAsBoolean(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey(), false), is(true)); assertThat(clusterState.routingTable().index(index), notNullValue()); assertThat(clusterState.blocks().hasIndexBlock(index, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); assertThat("Index " + index + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", @@ -320,7 +324,9 @@ static void assertIndexIsClosed(final String... indices) { static void assertIndexIsOpened(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); for (String index : indices) { - assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.OPEN)); + final IndexMetaData indexMetaData = clusterState.metaData().indices().get(index); + assertThat(indexMetaData.getState(), is(IndexMetaData.State.OPEN)); + assertThat(indexMetaData.getSettings().hasValue(MetaDataIndexStateService.VERIFIED_BEFORE_CLOSE_SETTING.getKey()), is(false)); assertThat(clusterState.routingTable().index(index), notNullValue()); assertThat(clusterState.blocks().hasIndexBlock(index, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(false)); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 2dc59ec4a43fd..76c356d59f7dd 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -945,7 +945,7 @@ public void testUpdateAnalysisLeaderIndexSettings() throws Exception { } assertBusy(() -> { - assertThat(getFollowTaskSettingsVersion("follower"), equalTo(2L)); + assertThat(getFollowTaskSettingsVersion("follower"), equalTo(4L)); assertThat(getFollowTaskMappingVersion("follower"), equalTo(2L)); GetSettingsRequest getSettingsRequest = new GetSettingsRequest(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java index 983b186c4ccf6..9231bad9a8dfe 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/index/engine/FrozenIndexTests.java @@ -349,7 +349,7 @@ public void testFreezeIndexIncreasesIndexSettingsVersion() throws ExecutionExcep assertAcked(xPackClient.freeze(new TransportFreezeIndexAction.FreezeRequest(index))); assertIndexFrozen(index); assertThat(client().admin().cluster().prepareState().get().getState().metaData().index(index).getSettingsVersion(), - equalTo(settingsVersion + 1)); + greaterThan(settingsVersion)); } public void testFreezeEmptyIndexWithTranslogOps() throws Exception { From 3e61939c7c7392904128fcdef8aaded90c835602 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 26 Feb 2019 14:13:44 +0100 Subject: [PATCH 16/20] Adapt CloseFollowerIndexIT for replicated closed indices (#38767) Now the test `CloseFollowerIndexIT` has been added in #38702, it needs to be adapted for replicated closed indices. The test closes the follower index which is lagging behind the leader index. When it's closed, no sanity checks are executed because it's a follower index (this is a consequence of #38702). But with replicated closed indices, the index is reinitialized as a closed index with a `NoOpEngine` and such engines make strong assertions on the values of the maximum sequence number and the global checkpoint. Since the values do not match, the shards cannot be created and fail and the cluster health turns RED. This commit adapts the `CloseFollowerIndexIT` test so that it wraps the default `UncaughtExceptionHandler` with a handler that tolerates any exception thrown by `ReadOnlyEngine.assertMaxSeqNoEqualsToGlobalCheckpoint()`. Replacing the default uncaught exception handler requires specific permissions, and instead of creating another gradle project it duplicates the `internalClusterTest` task to make it work without security manager for this specific test only. Relates to #33888 --- x-pack/plugin/ccr/build.gradle | 14 +++++ .../xpack/ccr/CloseFollowerIndexIT.java | 53 ++++++++++++++++--- 2 files changed, 60 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/ccr/build.gradle b/x-pack/plugin/ccr/build.gradle index b8ed9f55932cc..e1ddb2f12d78b 100644 --- a/x-pack/plugin/ccr/build.gradle +++ b/x-pack/plugin/ccr/build.gradle @@ -18,6 +18,9 @@ integTest.enabled = false compileJava.options.compilerArgs << "-Xlint:-try" compileTestJava.options.compilerArgs << "-Xlint:-try" +// Integration Test classes that cannot run with the security manager +String[] noSecurityManagerITClasses = [ "**/CloseFollowerIndexIT.class" ] + // Instead we create a separate task to run the // tests based on ESIntegTestCase task internalClusterTest(type: RandomizedTestingTask, @@ -25,11 +28,22 @@ task internalClusterTest(type: RandomizedTestingTask, description: 'Java fantasy integration tests', dependsOn: unitTest.dependsOn) { include '**/*IT.class' + exclude noSecurityManagerITClasses systemProperty 'es.set.netty.runtime.available.processors', 'false' } check.dependsOn internalClusterTest internalClusterTest.mustRunAfter test +task internalClusterTestNoSecurityManager(type: RandomizedTestingTask, + group: JavaBasePlugin.VERIFICATION_GROUP, + description: 'Java fantasy integration tests with no security manager', + dependsOn: unitTest.dependsOn) { + include noSecurityManagerITClasses + systemProperty 'es.set.netty.runtime.available.processors', 'false' + systemProperty 'tests.security.manager', 'false' +} +internalClusterTest.dependsOn internalClusterTestNoSecurityManager + // add all sub-projects of the qa sub-project gradle.projectsEvaluated { project.subprojects diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java index bd9f1a13bdb4d..2f111727b08ee 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java @@ -11,17 +11,21 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.xpack.CcrIntegTestCase; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; +import org.junit.After; +import org.junit.Before; -import java.util.ArrayList; -import java.util.List; +import java.security.AccessController; +import java.security.PrivilegedAction; import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonMap; @@ -31,7 +35,36 @@ public class CloseFollowerIndexIT extends CcrIntegTestCase { - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/38767") + private Thread.UncaughtExceptionHandler uncaughtExceptionHandler; + + @Before + public void wrapUncaughtExceptionHandler() { + uncaughtExceptionHandler = Thread.getDefaultUncaughtExceptionHandler(); + AccessController.doPrivileged((PrivilegedAction) () -> { + Thread.setDefaultUncaughtExceptionHandler((t, e) -> { + if (t.getThreadGroup().getName().contains(getTestClass().getSimpleName())) { + for (StackTraceElement element : e.getStackTrace()) { + if (element.getClassName().equals(ReadOnlyEngine.class.getName())) { + if (element.getMethodName().equals("assertMaxSeqNoEqualsToGlobalCheckpoint")) { + return; + } + } + } + } + uncaughtExceptionHandler.uncaughtException(t, e); + }); + return null; + }); + } + + @After + public void restoreUncaughtExceptionHandler() { + AccessController.doPrivileged((PrivilegedAction) () -> { + Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler); + return null; + }); + } + public void testCloseAndReopenFollowerIndex() throws Exception { final String leaderIndexSettings = getIndexSettings(1, 1, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); @@ -67,16 +100,22 @@ public void testCloseAndReopenFollowerIndex() throws Exception { assertThat(response.isAcknowledged(), is(true)); ClusterState clusterState = followerClient().admin().cluster().prepareState().get().getState(); - List blocks = new ArrayList<>(clusterState.getBlocks().indices().get("index2")); - assertThat(blocks.size(), equalTo(1)); - assertThat(blocks.get(0).id(), equalTo(MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID)); + assertThat(clusterState.metaData().index("index2").getState(), is(IndexMetaData.State.CLOSE)); + assertThat(clusterState.getBlocks().hasIndexBlock("index2", MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + assertThat(followerClient().admin().cluster().prepareHealth("index2").get().getStatus(), equalTo(ClusterHealthStatus.RED)); isRunning.set(false); for (Thread thread : threads) { thread.join(); } + assertAcked(followerClient().admin().indices().open(new OpenIndexRequest("index2")).get()); + clusterState = followerClient().admin().cluster().prepareState().get().getState(); + assertThat(clusterState.metaData().index("index2").getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.getBlocks().hasIndexBlockWithId("index2", MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID), is(false)); + ensureFollowerGreen("index2"); + refresh(leaderClient(), "index1"); SearchRequest leaderSearchRequest = new SearchRequest("index1"); leaderSearchRequest.source().trackTotalHits(true); From 5e7a4283169304d5987798fe45ff6534a545e4a6 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 26 Feb 2019 14:52:16 +0100 Subject: [PATCH 17/20] Adapt the Cluster Health API to closed indices (#39364) This commit adapts the Cluster Health API to support replicated closed indices. In order to do that, it removes the hard coded indices options from the `ClusterHealthRequest` and replaces it with a new `IndicesOptions.lenientExpand()` option. This option will be used by the master node (once it is upgraded to 8.0) to compute the global cluster health using both opened and closed indices information by default. The `expand_wildcards` REST parameter is also documented and tests where added to ensure that a specific expansion type can be used to monitoring the health of a only opened or only closed indices. Since the Cat Indices relies on the Cluster Health API, it has been adapted to report information about closed indices too. Note that the health and number of shards/replicas is only printed out for closed indices that have an index routing table. Closed indices without routing table have the same output as before. Related to #33888 --- .../rest-api-spec/api/cluster.health.json | 6 + .../test/cat.indices/10_basic.yml | 85 ++++++++-- .../test/cluster.health/10_basic.yml | 146 ++++++++++++++++ .../cluster.health/30_indices_options.yml | 79 +++++++++ .../cluster/health/ClusterHealthRequest.java | 16 +- .../health/ClusterHealthRequestBuilder.java | 6 + .../action/support/IndicesOptions.java | 14 +- .../cluster/RestClusterHealthAction.java | 6 +- .../rest/action/cat/RestIndicesAction.java | 60 ++++--- .../health/ClusterHealthRequestTests.java | 104 +++++++++++- .../cluster/ClusterHealthIT.java | 159 ++++++++++++++++++ .../IndexNameExpressionResolverTests.java | 7 +- 12 files changed, 641 insertions(+), 47 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/30_indices_options.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json index ee32a87c927ed..49d0ef5c83aee 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json @@ -12,6 +12,12 @@ } }, "params": { + "expand_wildcards": { + "type" : "enum", + "options" : ["open","closed","none","all"], + "default" : "all", + "description" : "Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, "level": { "type" : "enum", "options" : ["cluster","indices","shards"], diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yml index c7eddf42d1b03..9fb95c70c0b10 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.indices/10_basic.yml @@ -1,5 +1,5 @@ --- -"Test cat indices output": +"Test cat indices output (no indices)": - do: cat.indices: {} @@ -7,6 +7,8 @@ - match: $body: | /^$/ +--- +"Test cat indices output": - do: indices.create: @@ -47,29 +49,88 @@ (\d\d\d\d\-\d\d\-\d\dT\d\d:\d\d:\d\d.\d\d\dZ) \s* ) $/ +--- +"Test cat indices output for closed index (pre 8.0.0)": + - skip: + version: "8.0.0 - " + reason: "closed indices are replicated starting version 8.0" + + - do: + indices.create: + index: index-2 + body: + settings: + number_of_shards: 3 + number_of_replicas: 0 + - do: indices.close: - index: index1 + index: index-2 + - is_true: acknowledged + + - do: + cluster.health: + wait_for_status: green - do: cat.indices: - index: index* + index: index-* - match: $body: | - /^( \s+ - close \s+ - index1 \s+ + /^( \s+ + close \s+ + index-2 \s+ ([a-zA-Z0-9=/_+]|[\\\-]){22} \s+ - \s+ - \s+ - \s+ - \s+ - \s+ - \s* + \s+ + \s+ + \s+ + \s+ + \s+ + \s* ) $/ +--- +"Test cat indices output for closed index": + - skip: + version: " - 7.99.99" + reason: "closed indices are replicated starting version 8.0" + - do: + indices.create: + index: index-2 + body: + settings: + number_of_shards: 3 + number_of_replicas: 0 + + - do: + indices.close: + index: index-2 + - is_true: acknowledged + + - do: + cluster.health: + wait_for_status: green + + - do: + cat.indices: + index: index-* + + - match: + $body: | + /^(green \s+ + close \s+ + index-2 \s+ + ([a-zA-Z0-9=/_+]|[\\\-]){22} \s+ + 3 \s+ + 0 \s+ + \s+ + \s+ + \s+ + \s* + ) + $/ --- "Test cat indices using health status": diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/10_basic.yml index 70d7372234a51..73e162c80533b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/10_basic.yml @@ -132,4 +132,150 @@ - is_true: indices - is_true: indices.test_index.shards +--- +"cluster health with closed index (pre 8.0)": + - skip: + version: "8.0.0 - " + reason: "closed indices are replicated starting version 8.0" + + - do: + indices.create: + index: index-1 + body: + settings: + index: + number_of_replicas: 0 + + - do: + cluster.health: + wait_for_status: green + - match: { status: green } + + - do: + indices.create: + index: index-2 + body: + settings: + index: + number_of_replicas: 50 + + - do: + cluster.health: + wait_for_status: yellow + wait_for_no_relocating_shards: true + - match: { status: yellow } + + - do: + cluster.health: + index: index-* + - match: { status: yellow } + + - do: + cluster.health: + index: index-1 + - match: { status: green } + + - do: + cluster.health: + index: index-2 + - match: { status: yellow } + + - do: + indices.close: + index: index-2 + - is_true: acknowledged + + # closing the index-2 turns the cluster health back to green + - do: + cluster.health: + wait_for_status: green + - match: { status: green } + + - do: + cluster.health: + index: index-* + - match: { status: green } + + - do: + cluster.health: + index: index-1 + - match: { status: green } + + - do: + cluster.health: + index: index-2 + - match: { status: green } +--- +"cluster health with closed index": + - skip: + version: " - 7.99.99" + reason: "closed indices are replicated starting version 8.0" + + - do: + indices.create: + index: index-1 + body: + settings: + index: + number_of_replicas: 0 + + - do: + cluster.health: + wait_for_status: green + - match: { status: green } + + - do: + indices.create: + index: index-2 + body: + settings: + index: + number_of_replicas: 50 + + - do: + cluster.health: + wait_for_status: yellow + wait_for_no_relocating_shards: true + - match: { status: yellow } + + - do: + cluster.health: + index: index-* + - match: { status: yellow } + + - do: + cluster.health: + index: index-1 + - match: { status: green } + + - do: + cluster.health: + index: index-2 + - match: { status: yellow } + + # closing the index-2 does not change the cluster health with replicated closed indices + - do: + indices.close: + index: index-2 + - is_true: acknowledged + + - do: + cluster.health: + wait_for_status: yellow + - match: { status: yellow } + + - do: + cluster.health: + index: index-* + - match: { status: yellow } + + - do: + cluster.health: + index: index-1 + - match: { status: green } + + - do: + cluster.health: + index: index-2 + - match: { status: yellow } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/30_indices_options.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/30_indices_options.yml new file mode 100644 index 0000000000000..bb8f9d01e0007 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.health/30_indices_options.yml @@ -0,0 +1,79 @@ +setup: + + - do: + indices.create: + index: index-1 + body: + settings: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.create: + index: index-2 + body: + settings: + number_of_shards: 2 + number_of_replicas: 0 + + - do: + cluster.health: + wait_for_status: green + + - do: + indices.close: + index: index-2 + + - do: + cluster.health: + wait_for_status: green + +--- +"cluster health with expand_wildcards": + - skip: + version: " - 7.99.99" + reason: "indices options has been introduced in cluster health request starting version 8.0" + + - do: + cluster.health: + index: "index-*" + level: indices + expand_wildcards: open + - match: { status: green } + - match: { active_shards: 1 } + - match: { indices.index-1.status: green } + - match: { indices.index-1.active_shards: 1 } + - is_false: indices.index-2 + + - do: + cluster.health: + index: "index-*" + level: indices + expand_wildcards: closed + - match: { status: green } + - match: { active_shards: 2 } + - is_false: indices.index-1 + - match: { indices.index-2.status: green } + - match: { indices.index-2.active_shards: 2 } + + - do: + cluster.health: + index: "index-*" + level: indices + expand_wildcards: all + - match: { status: green } + - match: { active_shards: 3 } + - match: { indices.index-1.status: green } + - match: { indices.index-1.active_shards: 1 } + - match: { indices.index-2.status: green } + - match: { indices.index-2.active_shards: 2 } + + - do: + cluster.health: + index: "index-*" + level: indices + expand_wildcards: none + - match: { status: green } + - match: { active_shards: 0 } + - is_false: indices.index-1 + - is_false: indices.index-2 diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequest.java index 0b9bcbf11b9a1..03a8e52dd60ed 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequest.java @@ -39,6 +39,7 @@ public class ClusterHealthRequest extends MasterNodeReadRequest implements IndicesRequest.Replaceable { private String[] indices; + private IndicesOptions indicesOptions = IndicesOptions.lenientExpand(); private TimeValue timeout = new TimeValue(30, TimeUnit.SECONDS); private ClusterHealthStatus waitForStatus; private boolean waitForNoRelocatingShards = false; @@ -83,6 +84,11 @@ public ClusterHealthRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_6_2_0)) { waitForNoInitializingShards = in.readBoolean(); } + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + indicesOptions = IndicesOptions.readIndicesOptions(in); + } else { + indicesOptions = IndicesOptions.lenientExpandOpen(); + } } @Override @@ -115,6 +121,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_6_2_0)) { out.writeBoolean(waitForNoInitializingShards); } + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + indicesOptions.writeIndicesOptions(out); + } } @Override @@ -130,7 +139,12 @@ public ClusterHealthRequest indices(String... indices) { @Override public IndicesOptions indicesOptions() { - return IndicesOptions.lenientExpandOpen(); + return indicesOptions; + } + + public ClusterHealthRequest indicesOptions(final IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + return this; } public TimeValue timeout() { diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestBuilder.java index c79aac2afaf1a..21bf3357d5be4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestBuilder.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.cluster.health; import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.cluster.health.ClusterHealthStatus; @@ -38,6 +39,11 @@ public ClusterHealthRequestBuilder setIndices(String... indices) { return this; } + public ClusterHealthRequestBuilder setIndicesOptions(final IndicesOptions indicesOptions) { + request.indicesOptions(indicesOptions); + return this; + } + public ClusterHealthRequestBuilder setTimeout(TimeValue timeout) { request.timeout(timeout); return this; diff --git a/server/src/main/java/org/elasticsearch/action/support/IndicesOptions.java b/server/src/main/java/org/elasticsearch/action/support/IndicesOptions.java index c38c2a1c69a7f..81364c31c9ed9 100644 --- a/server/src/main/java/org/elasticsearch/action/support/IndicesOptions.java +++ b/server/src/main/java/org/elasticsearch/action/support/IndicesOptions.java @@ -90,7 +90,11 @@ public enum Option { public static final IndicesOptions STRICT_EXPAND_OPEN = new IndicesOptions(EnumSet.of(Option.ALLOW_NO_INDICES), EnumSet.of(WildcardStates.OPEN)); public static final IndicesOptions LENIENT_EXPAND_OPEN = - new IndicesOptions(EnumSet.of(Option.ALLOW_NO_INDICES, Option.IGNORE_UNAVAILABLE), EnumSet.of(WildcardStates.OPEN)); + new IndicesOptions(EnumSet.of(Option.ALLOW_NO_INDICES, Option.IGNORE_UNAVAILABLE), + EnumSet.of(WildcardStates.OPEN)); + public static final IndicesOptions LENIENT_EXPAND_OPEN_CLOSED = + new IndicesOptions(EnumSet.of(Option.ALLOW_NO_INDICES, Option.IGNORE_UNAVAILABLE), + EnumSet.of(WildcardStates.OPEN, WildcardStates.CLOSED)); public static final IndicesOptions STRICT_EXPAND_OPEN_CLOSED = new IndicesOptions(EnumSet.of(Option.ALLOW_NO_INDICES), EnumSet.of(WildcardStates.OPEN, WildcardStates.CLOSED)); public static final IndicesOptions STRICT_EXPAND_OPEN_FORBID_CLOSED = @@ -440,6 +444,14 @@ public static IndicesOptions lenientExpandOpen() { return LENIENT_EXPAND_OPEN; } + /** + * @return indices options that ignores unavailable indices, expands wildcards to both open and closed + * indices and allows that no indices are resolved from wildcard expressions (not returning an error). + */ + public static IndicesOptions lenientExpand() { + return LENIENT_EXPAND_OPEN_CLOSED; + } + @Override public boolean equals(Object obj) { if (obj == null) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java index 2f68cac9a0e1f..5e6f98eedd853 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.Priority; @@ -39,9 +40,9 @@ import static org.elasticsearch.client.Requests.clusterHealthRequest; public class RestClusterHealthAction extends BaseRestHandler { + public RestClusterHealthAction(Settings settings, RestController controller) { super(settings); - controller.registerHandler(RestRequest.Method.GET, "/_cluster/health", this); controller.registerHandler(RestRequest.Method.GET, "/_cluster/health/{index}", this); } @@ -53,7 +54,8 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - ClusterHealthRequest clusterHealthRequest = clusterHealthRequest(Strings.splitStringByCommaToArray(request.param("index"))); + final ClusterHealthRequest clusterHealthRequest = clusterHealthRequest(Strings.splitStringByCommaToArray(request.param("index"))); + clusterHealthRequest.indicesOptions(IndicesOptions.fromRequest(request, clusterHealthRequest.indicesOptions())); clusterHealthRequest.local(request.paramAsBoolean("local", clusterHealthRequest.local())); clusterHealthRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterHealthRequest.masterNodeTimeout())); clusterHealthRequest.timeout(request.paramAsTime("timeout", clusterHealthRequest.timeout())); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index 9b672a9992cc8..24f3ac8522aa2 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -104,9 +104,9 @@ public void processResponse(final ClusterStateResponse clusterStateResponse) { // 2) the deleted index was resolved as part of a wildcard or _all. In this case, we want the subsequent requests not to // fail on the deleted index (as we want to ignore wildcards that cannot be resolved). // This behavior can be ensured by letting the cluster health and indices stats requests re-resolve the index names with the - // same indices options that we used for the initial cluster state request (strictExpand). Unfortunately cluster health - // requests hard-code their indices options and the best we can do is apply strictExpand to the indices stats request. + // same indices options that we used for the initial cluster state request (strictExpand). final ClusterHealthRequest clusterHealthRequest = Requests.clusterHealthRequest(indices); + clusterHealthRequest.indicesOptions(strictExpandIndicesOptions); clusterHealthRequest.local(request.paramAsBoolean("local", clusterHealthRequest.local())); client.admin().cluster().health(clusterHealthRequest, new RestActionListener(channel) { @@ -383,34 +383,37 @@ protected Table getTableWithHeader(final RestRequest request) { } // package private for testing - Table buildTable(RestRequest request, IndexMetaData[] indicesMetaData, ClusterHealthResponse response, IndicesStatsResponse stats) { + Table buildTable(final RestRequest request, + final IndexMetaData[] indicesMetaData, + final ClusterHealthResponse clusterHealthResponse, + final IndicesStatsResponse indicesStatsResponse) { final String healthParam = request.param("health"); - final ClusterHealthStatus status; - if (healthParam != null) { - status = ClusterHealthStatus.fromString(healthParam); - } else { - status = null; - } - - Table table = getTableWithHeader(request); + final Table table = getTableWithHeader(request); for (IndexMetaData indexMetaData : indicesMetaData) { final String indexName = indexMetaData.getIndex().getName(); - ClusterIndexHealth indexHealth = response.getIndices().get(indexName); - IndexStats indexStats = stats.getIndices().get(indexName); - IndexMetaData.State state = indexMetaData.getState(); - boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(indexMetaData.getSettings()); - - if (status != null) { - if (state == IndexMetaData.State.CLOSE || - (indexHealth == null && false == ClusterHealthStatus.RED.equals(status)) || - false == indexHealth.getStatus().equals(status)) { + final ClusterIndexHealth indexHealth = clusterHealthResponse.getIndices().get(indexName); + final IndexStats indexStats = indicesStatsResponse.getIndices().get(indexName); + final IndexMetaData.State indexState = indexMetaData.getState(); + final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(indexMetaData.getSettings()); + + if (healthParam != null) { + final ClusterHealthStatus healthStatusFilter = ClusterHealthStatus.fromString(healthParam); + boolean skip; + if (indexHealth != null) { + // index health is known but does not match the one requested + skip = indexHealth.getStatus() != healthStatusFilter; + } else { + // index health is unknown, skip if we don't explicitly request RED health or if the index is closed but not replicated + skip = ClusterHealthStatus.RED != healthStatusFilter || indexState == IndexMetaData.State.CLOSE; + } + if (skip) { continue; } } // the open index is present in the cluster state but is not returned in the indices stats API - if (indexStats == null && state != IndexMetaData.State.CLOSE) { + if (indexStats == null && indexState != IndexMetaData.State.CLOSE) { // the index stats API is called last, after cluster state and cluster health. If the index stats // has not resolved the same open indices as the initial cluster state call, then the indices might // have been removed in the meantime or, more likely, are unauthorized. This is because the cluster @@ -422,9 +425,8 @@ Table buildTable(RestRequest request, IndexMetaData[] indicesMetaData, ClusterHe final CommonStats primaryStats; final CommonStats totalStats; - if (state == IndexMetaData.State.CLOSE) { + if (indexState == IndexMetaData.State.CLOSE) { // empty stats for closed indices, but their names are displayed - assert indexStats == null; primaryStats = new CommonStats(); totalStats = new CommonStats(); } else { @@ -433,9 +435,15 @@ Table buildTable(RestRequest request, IndexMetaData[] indicesMetaData, ClusterHe } table.startRow(); - table.addCell(state == IndexMetaData.State.OPEN ? - (indexHealth == null ? "red*" : indexHealth.getStatus().toString().toLowerCase(Locale.ROOT)) : null); - table.addCell(state.toString().toLowerCase(Locale.ROOT)); + + String health = null; + if (indexHealth != null) { + health = indexHealth.getStatus().toString().toLowerCase(Locale.ROOT); + } else if (indexStats != null) { + health = "red*"; + } + table.addCell(health); + table.addCell(indexState.toString().toLowerCase(Locale.ROOT)); table.addCell(indexName); table.addCell(indexMetaData.getIndexUUID()); table.addCell(indexHealth == null ? null : indexHealth.getNumberOfShards()); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestTests.java index 1e85005add0e9..75fa5a2393d37 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/health/ClusterHealthRequestTests.java @@ -19,15 +19,23 @@ package org.elasticsearch.action.admin.cluster.health; +import org.elasticsearch.Version; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; +import java.util.Locale; + +import static org.elasticsearch.test.VersionUtils.getPreviousVersion; +import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.hamcrest.core.IsEqual.equalTo; public class ClusterHealthRequestTests extends ESTestCase { + public void testSerialize() throws Exception { final ClusterHealthRequest originalRequest = randomRequest(); final ClusterHealthRequest cloneRequest; @@ -43,9 +51,89 @@ public void testSerialize() throws Exception { assertThat(cloneRequest.waitForNoRelocatingShards(), equalTo(originalRequest.waitForNoRelocatingShards())); assertThat(cloneRequest.waitForActiveShards(), equalTo(originalRequest.waitForActiveShards())); assertThat(cloneRequest.waitForEvents(), equalTo(originalRequest.waitForEvents())); + assertIndicesEquals(cloneRequest.indices(), originalRequest.indices()); + assertThat(cloneRequest.indicesOptions(), equalTo(originalRequest.indicesOptions())); + } + + public void testBwcSerialization() throws Exception { + for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { + // Generate a random cluster health request in version < 8.0.0 and serializes it + final BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(randomVersionBetween(random(), Version.V_6_3_0, getPreviousVersion(Version.V_8_0_0))); + + final ClusterHealthRequest expected = randomRequest(); + { + expected.getParentTask().writeTo(out); + out.writeTimeValue(expected.masterNodeTimeout()); + out.writeBoolean(expected.local()); + if (expected.indices() == null) { + out.writeVInt(0); + } else { + out.writeVInt(expected.indices().length); + for (String index : expected.indices()) { + out.writeString(index); + } + } + out.writeTimeValue(expected.timeout()); + if (expected.waitForStatus() == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeByte(expected.waitForStatus().value()); + } + out.writeBoolean(expected.waitForNoRelocatingShards()); + expected.waitForActiveShards().writeTo(out); + out.writeString(expected.waitForNodes()); + if (expected.waitForEvents() == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + Priority.writeTo(expected.waitForEvents(), out); + } + out.writeBoolean(expected.waitForNoInitializingShards()); + } + + // Deserialize and check the cluster health request + final StreamInput in = out.bytes().streamInput(); + in.setVersion(out.getVersion()); + final ClusterHealthRequest actual = new ClusterHealthRequest(in); + + assertThat(actual.waitForStatus(), equalTo(expected.waitForStatus())); + assertThat(actual.waitForNodes(), equalTo(expected.waitForNodes())); + assertThat(actual.waitForNoInitializingShards(), equalTo(expected.waitForNoInitializingShards())); + assertThat(actual.waitForNoRelocatingShards(), equalTo(expected.waitForNoRelocatingShards())); + assertThat(actual.waitForActiveShards(), equalTo(expected.waitForActiveShards())); + assertThat(actual.waitForEvents(), equalTo(expected.waitForEvents())); + assertIndicesEquals(actual.indices(), expected.indices()); + assertThat(actual.indicesOptions(), equalTo(IndicesOptions.lenientExpandOpen())); + } + + for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { + // Generate a random cluster health request in current version + final ClusterHealthRequest expected = randomRequest(); + + // Serialize to node in version < 8.0.0 + final BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(randomVersionBetween(random(), Version.V_6_3_0, getPreviousVersion(Version.V_8_0_0))); + expected.writeTo(out); + + // Deserialize and check the cluster health request + final StreamInput in = out.bytes().streamInput(); + in.setVersion(out.getVersion()); + final ClusterHealthRequest actual = new ClusterHealthRequest(in); + + assertThat(actual.waitForStatus(), equalTo(expected.waitForStatus())); + assertThat(actual.waitForNodes(), equalTo(expected.waitForNodes())); + assertThat(actual.waitForNoInitializingShards(), equalTo(expected.waitForNoInitializingShards())); + assertThat(actual.waitForNoRelocatingShards(), equalTo(expected.waitForNoRelocatingShards())); + assertThat(actual.waitForActiveShards(), equalTo(expected.waitForActiveShards())); + assertThat(actual.waitForEvents(), equalTo(expected.waitForEvents())); + assertIndicesEquals(actual.indices(), expected.indices()); + assertThat(actual.indicesOptions(), equalTo(IndicesOptions.lenientExpandOpen())); + } } - ClusterHealthRequest randomRequest() { + private ClusterHealthRequest randomRequest() { ClusterHealthRequest request = new ClusterHealthRequest(); request.waitForStatus(randomFrom(ClusterHealthStatus.values())); request.waitForNodes(randomFrom("", "<", "<=", ">", ">=") + between(0, 1000)); @@ -53,7 +141,21 @@ ClusterHealthRequest randomRequest() { request.waitForNoRelocatingShards(randomBoolean()); request.waitForActiveShards(randomIntBetween(0, 10)); request.waitForEvents(randomFrom(Priority.values())); + if (randomBoolean()) { + final String[] indices = new String[randomIntBetween(1, 10)]; + for (int i = 0; i < indices.length; i++) { + indices[i] = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + } + request.indices(indices); + } + if (randomBoolean()) { + request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); + } return request; } + private static void assertIndicesEquals(final String[] actual, final String[] expected) { + // null indices in ClusterHealthRequest is deserialized as empty string array + assertArrayEquals(expected != null ? expected : Strings.EMPTY_ARRAY, actual); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterHealthIT.java b/server/src/test/java/org/elasticsearch/cluster/ClusterHealthIT.java index 8693308e8650b..d0680e91b8b73 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterHealthIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterHealthIT.java @@ -20,13 +20,19 @@ package org.elasticsearch.cluster; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; + import java.util.concurrent.atomic.AtomicBoolean; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; public class ClusterHealthIT extends ESIntegTestCase { @@ -76,6 +82,159 @@ public void testHealth() { assertThat(healthResponse.getIndices().size(), equalTo(1)); } + public void testHealthWithClosedIndices() { + createIndex("index-1"); + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth().setWaitForGreenStatus().get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + + createIndex("index-2"); + assertAcked(client().admin().indices().prepareClose("index-2")); + + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus() + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(2)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-1").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-2").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(2)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*") + .setIndicesOptions(IndicesOptions.lenientExpandOpen()) + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2"), nullValue()); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*") + .setIndicesOptions(IndicesOptions.fromOptions(true, true, false, true)) + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-1"), nullValue()); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + + createIndex("index-3", Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 50) + .build()); + assertAcked(client().admin().indices().prepareClose("index-3")); + + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth() + .setWaitForNoRelocatingShards(true) + .setWaitForNoInitializingShards(true) + .setWaitForYellowStatus() + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(3)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-3").getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-1").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-2").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-3").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-3").getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*").get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(3)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-3").getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*") + .setIndicesOptions(IndicesOptions.lenientExpandOpen()) + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(1)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2"), nullValue()); + assertThat(response.getIndices().get("index-3"), nullValue()); + } + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth("index-*") + .setIndicesOptions(IndicesOptions.fromOptions(true, true, false, true)) + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(2)); + assertThat(response.getIndices().get("index-1"), nullValue()); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-3").getStatus(), equalTo(ClusterHealthStatus.YELLOW)); + } + + assertAcked(client().admin().indices().prepareUpdateSettings("index-3") + .setSettings(Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas()) + .build())); + { + ClusterHealthResponse response = client().admin().cluster().prepareHealth() + .setWaitForGreenStatus() + .get(); + assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.isTimedOut(), equalTo(false)); + assertThat(response.getIndices().size(), equalTo(3)); + assertThat(response.getIndices().get("index-1").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-2").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.getIndices().get("index-3").getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + } + public void testHealthOnIndexCreation() throws Exception { final AtomicBoolean finished = new AtomicBoolean(false); Thread clusterHealthThread = new Thread() { diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java index 5c96f6f1cbcd4..571843126f98c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -155,8 +155,7 @@ public void testIndexOptionsLenient() { .put(indexBuilder("foofoo").putAlias(AliasMetaData.builder("barbaz"))); ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build(); - IndicesOptions lenientExpand = IndicesOptions.fromOptions(true, true, true, true); - IndicesOptions[] indicesOptions = new IndicesOptions[]{ IndicesOptions.lenientExpandOpen(), lenientExpand}; + IndicesOptions[] indicesOptions = new IndicesOptions[]{IndicesOptions.lenientExpandOpen(), IndicesOptions.lenientExpand()}; for (IndicesOptions options : indicesOptions) { IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo"); @@ -199,7 +198,7 @@ public void testIndexOptionsLenient() { String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); - context = new IndexNameExpressionResolver.Context(state, lenientExpand); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand()); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(Arrays.toString(results), 4, results.length); @@ -208,7 +207,7 @@ public void testIndexOptionsLenient() { assertEquals(3, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo")); - context = new IndexNameExpressionResolver.Context(state, lenientExpand); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand()); results = indexNameExpressionResolver.concreteIndexNames(context, "foofoo*"); assertEquals(4, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo", "foofoo-closed")); From 3f9993dc262b50e550f91042ca2416bbb28e5dcb Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 26 Feb 2019 18:34:03 +0100 Subject: [PATCH 18/20] Wait for shards to be active after closing indices (#38854) This commit changes the Close Index API to add a `wait_for_active_shards` parameter that allows to wait for shards of closed indices to be active before returning a response. Relates #33888 --- .../rest-api-spec/api/indices.close.json | 4 + .../test/indices.open/10_basic.yml | 24 ++++ .../test/indices.open/20_multiple_indices.yml | 6 + .../admin/indices/close/CloseIndexAction.java | 7 +- .../CloseIndexClusterStateUpdateRequest.java | 18 ++- .../indices/close/CloseIndexRequest.java | 20 +++ .../close/CloseIndexRequestBuilder.java | 31 ++++- .../indices/close/CloseIndexResponse.java | 52 ++++++++ .../close/TransportCloseIndexAction.java | 27 +++-- .../client/IndicesAdminClient.java | 5 +- .../client/support/AbstractClient.java | 5 +- .../metadata/MetaDataIndexStateService.java | 29 ++++- .../admin/indices/RestCloseIndexAction.java | 5 + .../indices/close/CloseIndexRequestTests.java | 114 ++++++++++++++++++ .../close/CloseIndexResponseTests.java | 86 +++++++++++++ .../index/shard/IndexShardIT.java | 4 +- .../indices/IndicesLifecycleListenerIT.java | 10 +- .../indices/state/CloseIndexIT.java | 26 +++- .../indices/state/ReopenWhileClosingIT.java | 6 +- .../indices/state/SimpleIndexStateIT.java | 2 +- .../ccr/action/ShardFollowTasksExecutor.java | 3 +- .../xpack/ccr/CloseFollowerIndexIT.java | 8 +- .../action/TransportFreezeIndexAction.java | 5 +- .../CloseFollowerIndexStepTests.java | 6 +- 24 files changed, 459 insertions(+), 44 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponseTests.java diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json index 4eaa93030ee7b..55fd245f26c91 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json @@ -34,6 +34,10 @@ "options" : ["open","closed","none","all"], "default" : "open", "description" : "Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, + "wait_for_active_shards": { + "type" : "string", + "description" : "Sets the number of active shards to wait for before the operation returns." } } }, diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/10_basic.yml index 64e59d5939287..a389fee9bf761 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/10_basic.yml @@ -14,6 +14,7 @@ - do: indices.close: index: test_index + - is_true: acknowledged - do: catch: bad_request @@ -24,6 +25,7 @@ - do: indices.open: index: test_index + - is_true: acknowledged - do: cluster.health: @@ -50,11 +52,33 @@ - do: indices.close: index: test_index + - is_true: acknowledged - do: indices.open: index: test_index wait_for_active_shards: all + - is_true: acknowledged + - match: { acknowledged: true } + - match: { shards_acknowledged: true } + +--- +"Close index with wait_for_active_shards set to all": + - skip: + version: " - 7.99.99" + reason: "closed indices are replicated starting version 8.0" + + - do: + indices.create: + index: test_index + body: + settings: + number_of_replicas: 0 + - do: + indices.close: + index: test_index + wait_for_active_shards: all + - is_true: acknowledged - match: { acknowledged: true } - match: { shards_acknowledged: true } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/20_multiple_indices.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/20_multiple_indices.yml index 8e1bf660f6378..bef5ea8a54651 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/20_multiple_indices.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.open/20_multiple_indices.yml @@ -26,6 +26,7 @@ setup: - do: indices.close: index: _all + - is_true: acknowledged - do: catch: bad_request @@ -36,6 +37,7 @@ setup: - do: indices.open: index: _all + - is_true: acknowledged - do: cluster.health: @@ -51,6 +53,7 @@ setup: - do: indices.close: index: test_* + - is_true: acknowledged - do: catch: bad_request @@ -61,6 +64,7 @@ setup: - do: indices.open: index: test_* + - is_true: acknowledged - do: cluster.health: @@ -76,6 +80,7 @@ setup: - do: indices.close: index: '*' + - is_true: acknowledged - do: catch: bad_request @@ -86,6 +91,7 @@ setup: - do: indices.open: index: '*' + - is_true: acknowledged - do: cluster.health: diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java index 68a911ff58627..5c3d60dd44013 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexAction.java @@ -20,9 +20,8 @@ package org.elasticsearch.action.admin.indices.close; import org.elasticsearch.action.Action; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -public class CloseIndexAction extends Action { +public class CloseIndexAction extends Action { public static final CloseIndexAction INSTANCE = new CloseIndexAction(); public static final String NAME = "indices:admin/close"; @@ -32,7 +31,7 @@ private CloseIndexAction() { } @Override - public AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + public CloseIndexResponse newResponse() { + return new CloseIndexResponse(); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java index bb0f98ac07b7e..955ddf6fe8a76 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.admin.indices.close; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ack.IndicesClusterStateUpdateRequest; /** @@ -25,7 +26,8 @@ */ public class CloseIndexClusterStateUpdateRequest extends IndicesClusterStateUpdateRequest { - private final long taskId; + private long taskId; + private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT; public CloseIndexClusterStateUpdateRequest(final long taskId) { this.taskId = taskId; @@ -34,4 +36,18 @@ public CloseIndexClusterStateUpdateRequest(final long taskId) { public long taskId() { return taskId; } + + public CloseIndexClusterStateUpdateRequest taskId(final long taskId) { + this.taskId = taskId; + return this; + } + + public ActiveShardCount waitForActiveShards() { + return waitForActiveShards; + } + + public CloseIndexClusterStateUpdateRequest waitForActiveShards(final ActiveShardCount waitForActiveShards) { + this.waitForActiveShards = waitForActiveShards; + return this; + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java index 272bae9425712..e7979beb68214 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java @@ -19,8 +19,10 @@ package org.elasticsearch.action.admin.indices.close; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.common.io.stream.StreamInput; @@ -38,6 +40,7 @@ public class CloseIndexRequest extends AcknowledgedRequest im private String[] indices; private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen(); + private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT; //NORELEASE Changes this to NONE to keep previous behavior public CloseIndexRequest() { } @@ -101,11 +104,25 @@ public CloseIndexRequest indicesOptions(IndicesOptions indicesOptions) { return this; } + public ActiveShardCount waitForActiveShards() { + return waitForActiveShards; + } + + public CloseIndexRequest waitForActiveShards(final ActiveShardCount waitForActiveShards) { + this.waitForActiveShards = waitForActiveShards; + return this; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); indices = in.readStringArray(); indicesOptions = IndicesOptions.readIndicesOptions(in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + waitForActiveShards = ActiveShardCount.readFrom(in); + } else { + waitForActiveShards = ActiveShardCount.NONE; + } } @Override @@ -113,5 +130,8 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeStringArray(indices); indicesOptions.writeIndicesOptions(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + waitForActiveShards.writeTo(out); + } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestBuilder.java index e69c6fed87dcd..7db79e0c3e550 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestBuilder.java @@ -19,16 +19,16 @@ package org.elasticsearch.action.admin.indices.close; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedRequestBuilder; -import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.ElasticsearchClient; /** * Builder for close index request */ public class CloseIndexRequestBuilder - extends AcknowledgedRequestBuilder { + extends AcknowledgedRequestBuilder { public CloseIndexRequestBuilder(ElasticsearchClient client, CloseIndexAction action) { super(client, action, new CloseIndexRequest()); @@ -60,4 +60,31 @@ public CloseIndexRequestBuilder setIndicesOptions(IndicesOptions indicesOptions) request.indicesOptions(indicesOptions); return this; } + + /** + * Sets the number of shard copies that should be active for indices closing to return. + * Defaults to {@link ActiveShardCount#DEFAULT}, which will wait for one shard copy + * (the primary) to become active. Set this value to {@link ActiveShardCount#ALL} to + * wait for all shards (primary and all replicas) to be active before returning. + * Otherwise, use {@link ActiveShardCount#from(int)} to set this value to any + * non-negative integer, up to the number of copies per shard (number of replicas + 1), + * to wait for the desired amount of shard copies to become active before returning. + * Indices closing will only wait up until the timeout value for the number of shard copies + * to be active before returning. + * + * @param waitForActiveShards number of active shard copies to wait on + */ + public CloseIndexRequestBuilder setWaitForActiveShards(final ActiveShardCount waitForActiveShards) { + request.waitForActiveShards(waitForActiveShards); + return this; + } + + /** + * A shortcut for {@link #setWaitForActiveShards(ActiveShardCount)} where the numerical + * shard count is passed in, instead of having to first call {@link ActiveShardCount#from(int)} + * to get the ActiveShardCount. + */ + public CloseIndexRequestBuilder setWaitForActiveShards(final int waitForActiveShards) { + return setWaitForActiveShards(ActiveShardCount.from(waitForActiveShards)); + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java new file mode 100644 index 0000000000000..189712f0fca78 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponse.java @@ -0,0 +1,52 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.action.admin.indices.close; + +import org.elasticsearch.Version; +import org.elasticsearch.action.support.master.ShardsAcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class CloseIndexResponse extends ShardsAcknowledgedResponse { + + CloseIndexResponse() { + } + + public CloseIndexResponse(final boolean acknowledged, final boolean shardsAcknowledged) { + super(acknowledged, shardsAcknowledged); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + readShardsAcknowledged(in); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + writeShardsAcknowledged(out); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java index bb3db084b0c53..05f680af57ddf 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java @@ -23,7 +23,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.DestructiveOperations; -import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; @@ -44,7 +43,7 @@ /** * Close index action */ -public class TransportCloseIndexAction extends TransportMasterNodeAction { +public class TransportCloseIndexAction extends TransportMasterNodeAction { private final MetaDataIndexStateService indexStateService; private final DestructiveOperations destructiveOperations; @@ -76,12 +75,12 @@ protected String executor() { } @Override - protected AcknowledgedResponse newResponse() { - return new AcknowledgedResponse(); + protected CloseIndexResponse newResponse() { + return new CloseIndexResponse(); } @Override - protected void doExecute(Task task, CloseIndexRequest request, ActionListener listener) { + protected void doExecute(Task task, CloseIndexRequest request, ActionListener listener) { destructiveOperations.failDestructive(request.indices()); if (closeIndexEnabled == false) { throw new IllegalStateException("closing indices is disabled - set [" + CLUSTER_INDICES_CLOSE_ENABLE_SETTING.getKey() + @@ -97,29 +96,33 @@ protected ClusterBlockException checkBlock(CloseIndexRequest request, ClusterSta } @Override - protected void masterOperation(final CloseIndexRequest request, final ClusterState state, - final ActionListener listener) { + protected void masterOperation(final CloseIndexRequest request, + final ClusterState state, + final ActionListener listener) { throw new UnsupportedOperationException("The task parameter is required"); } @Override - protected void masterOperation(final Task task, final CloseIndexRequest request, final ClusterState state, - final ActionListener listener) throws Exception { + protected void masterOperation(final Task task, + final CloseIndexRequest request, + final ClusterState state, + final ActionListener listener) throws Exception { final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); if (concreteIndices == null || concreteIndices.length == 0) { - listener.onResponse(new AcknowledgedResponse(true)); + listener.onResponse(new CloseIndexResponse(true, false)); return; } final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest(task.getId()) .ackTimeout(request.timeout()) .masterNodeTimeout(request.masterNodeTimeout()) + .waitForActiveShards(request.waitForActiveShards()) .indices(concreteIndices); - indexStateService.closeIndices(closeRequest, new ActionListener() { + indexStateService.closeIndices(closeRequest, new ActionListener() { @Override - public void onResponse(final AcknowledgedResponse response) { + public void onResponse(final CloseIndexResponse response) { listener.onResponse(response); } diff --git a/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index 718dde98a0f97..d5a73981f29f1 100644 --- a/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/server/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -36,6 +36,7 @@ import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; @@ -307,7 +308,7 @@ public interface IndicesAdminClient extends ElasticsearchClient { * @return The result future * @see org.elasticsearch.client.Requests#closeIndexRequest(String) */ - ActionFuture close(CloseIndexRequest request); + ActionFuture close(CloseIndexRequest request); /** * Closes an index based on the index name. @@ -316,7 +317,7 @@ public interface IndicesAdminClient extends ElasticsearchClient { * @param listener A listener to be notified with a result * @see org.elasticsearch.client.Requests#closeIndexRequest(String) */ - void close(CloseIndexRequest request, ActionListener listener); + void close(CloseIndexRequest request, ActionListener listener); /** * Closes one or more indices based on their index name. diff --git a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java index 3fc931a85c0f7..e79f0567babe6 100644 --- a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -152,6 +152,7 @@ import org.elasticsearch.action.admin.indices.close.CloseIndexAction; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexRequestBuilder; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexAction; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; @@ -1355,12 +1356,12 @@ public DeleteIndexRequestBuilder prepareDelete(String... indices) { } @Override - public ActionFuture close(final CloseIndexRequest request) { + public ActionFuture close(final CloseIndexRequest request) { return execute(CloseIndexAction.INSTANCE, request); } @Override - public void close(final CloseIndexRequest request, final ActionListener listener) { + public void close(final CloseIndexRequest request, final ActionListener listener) { execute(CloseIndexAction.INSTANCE, request, listener); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index 4d81bf6e9c557..7c582483e3b42 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NotifyOnceListener; import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction; import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest; import org.elasticsearch.action.support.ActiveShardsObserver; @@ -123,7 +124,7 @@ public MetaDataIndexStateService(ClusterService clusterService, AllocationServic * Closing indices is a 3 steps process: it first adds a write block to every indices to close, then waits for the operations on shards * to be terminated and finally closes the indices by moving their state to CLOSE. */ - public void closeIndices(final CloseIndexClusterStateUpdateRequest request, final ActionListener listener) { + public void closeIndices(final CloseIndexClusterStateUpdateRequest request, final ActionListener listener) { final Index[] concreteIndices = request.indices(); if (concreteIndices == null || concreteIndices.length == 0) { throw new IllegalArgumentException("Index name is required"); @@ -143,7 +144,7 @@ public ClusterState execute(final ClusterState currentState) { public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { if (oldState == newState) { assert blockedIndices.isEmpty() : "List of blocked indices is not empty but cluster state wasn't changed"; - listener.onResponse(new AcknowledgedResponse(true)); + listener.onResponse(new CloseIndexResponse(true, false)); } else { assert blockedIndices.isEmpty() == false : "List of blocked indices is empty but cluster state was changed"; threadPool.executor(ThreadPool.Names.MANAGEMENT) @@ -174,7 +175,29 @@ public void onFailure(final String source, final Exception e) { @Override public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { - listener.onResponse(new AcknowledgedResponse(acknowledged)); + + final String[] indices = results.entrySet().stream() + .filter(result -> result.getValue().isAcknowledged()) + .map(result -> result.getKey().getName()) + .filter(index -> newState.routingTable().hasIndex(index)) + .toArray(String[]::new); + + if (indices.length > 0) { + activeShardsObserver.waitForActiveShards(indices, request.waitForActiveShards(), + request.ackTimeout(), shardsAcknowledged -> { + if (shardsAcknowledged == false) { + logger.debug("[{}] indices closed, but the operation timed out while waiting " + + "for enough shards to be started.", Arrays.toString(indices)); + } + // acknowledged maybe be false but some indices may have been correctly closed, so + // we maintain a kind of coherency by overriding the shardsAcknowledged value + // (see ShardsAcknowledgedResponse constructor) + boolean shardsAcked = acknowledged ? shardsAcknowledged : false; + listener.onResponse(new CloseIndexResponse(acknowledged, shardsAcked)); + }, listener::onFailure); + } else { + listener.onResponse(new CloseIndexResponse(acknowledged, false)); + } } }), listener::onFailure) diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCloseIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCloseIndexAction.java index b2475cafcbeb6..3ee2687eb7288 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCloseIndexAction.java @@ -20,6 +20,7 @@ package org.elasticsearch.rest.action.admin.indices; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; @@ -49,6 +50,10 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC closeIndexRequest.masterNodeTimeout(request.paramAsTime("master_timeout", closeIndexRequest.masterNodeTimeout())); closeIndexRequest.timeout(request.paramAsTime("timeout", closeIndexRequest.timeout())); closeIndexRequest.indicesOptions(IndicesOptions.fromRequest(request, closeIndexRequest.indicesOptions())); + String waitForActiveShards = request.param("wait_for_active_shards"); + if (waitForActiveShards != null) { + closeIndexRequest.waitForActiveShards(ActiveShardCount.parseString(waitForActiveShards)); + } return channel -> client.admin().indices().close(closeIndexRequest, new RestToXContentListener<>(channel)); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestTests.java new file mode 100644 index 0000000000000..53b39027a697b --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequestTests.java @@ -0,0 +1,114 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.close; + +import org.elasticsearch.Version; +import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; + +import static org.elasticsearch.test.VersionUtils.randomVersionBetween; + +public class CloseIndexRequestTests extends ESTestCase { + + public void testSerialization() throws Exception { + final CloseIndexRequest request = randomRequest(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + request.writeTo(out); + + final CloseIndexRequest deserializedRequest = new CloseIndexRequest(); + try (StreamInput in = out.bytes().streamInput()) { + deserializedRequest.readFrom(in); + } + assertEquals(request.timeout(), deserializedRequest.timeout()); + assertEquals(request.masterNodeTimeout(), deserializedRequest.masterNodeTimeout()); + assertEquals(request.indicesOptions(), deserializedRequest.indicesOptions()); + assertEquals(request.getParentTask(), deserializedRequest.getParentTask()); + assertEquals(request.waitForActiveShards(), deserializedRequest.waitForActiveShards()); + assertArrayEquals(request.indices(), deserializedRequest.indices()); + } + } + + public void testBwcSerialization() throws Exception { + { + final CloseIndexRequest request = randomRequest(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.setVersion(randomVersionBetween(random(), Version.V_6_4_0, VersionUtils.getPreviousVersion(Version.V_8_0_0))); + request.writeTo(out); + + try (StreamInput in = out.bytes().streamInput()) { + assertEquals(request.getParentTask(), TaskId.readFromStream(in)); + assertEquals(request.masterNodeTimeout(), in.readTimeValue()); + assertEquals(request.timeout(), in.readTimeValue()); + assertArrayEquals(request.indices(), in.readStringArray()); + assertEquals(request.indicesOptions(), IndicesOptions.readIndicesOptions(in)); + } + } + } + { + final CloseIndexRequest sample = randomRequest(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + sample.getParentTask().writeTo(out); + out.writeTimeValue(sample.masterNodeTimeout()); + out.writeTimeValue(sample.timeout()); + out.writeStringArray(sample.indices()); + sample.indicesOptions().writeIndicesOptions(out); + + final CloseIndexRequest deserializedRequest = new CloseIndexRequest(); + try (StreamInput in = out.bytes().streamInput()) { + in.setVersion(randomVersionBetween(random(), Version.V_6_4_0, VersionUtils.getPreviousVersion(Version.V_8_0_0))); + deserializedRequest.readFrom(in); + } + assertEquals(sample.getParentTask(), deserializedRequest.getParentTask()); + assertEquals(sample.masterNodeTimeout(), deserializedRequest.masterNodeTimeout()); + assertEquals(sample.timeout(), deserializedRequest.timeout()); + assertArrayEquals(sample.indices(), deserializedRequest.indices()); + assertEquals(sample.indicesOptions(), deserializedRequest.indicesOptions()); + assertEquals(ActiveShardCount.NONE, deserializedRequest.waitForActiveShards()); + } + } + } + + private CloseIndexRequest randomRequest() { + CloseIndexRequest request = new CloseIndexRequest(); + request.indices(generateRandomStringArray(10, 5, false, false)); + if (randomBoolean()) { + request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); + } + if (randomBoolean()) { + request.timeout(randomPositiveTimeValue()); + } + if (randomBoolean()) { + request.masterNodeTimeout(randomPositiveTimeValue()); + } + if (randomBoolean()) { + request.setParentTask(randomAlphaOfLength(5), randomNonNegativeLong()); + } + if (randomBoolean()) { + request.waitForActiveShards(randomFrom(ActiveShardCount.DEFAULT, ActiveShardCount.NONE, ActiveShardCount.ONE, + ActiveShardCount.ALL)); + } + return request; + } +} diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponseTests.java new file mode 100644 index 0000000000000..dc859cfab63a9 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/CloseIndexResponseTests.java @@ -0,0 +1,86 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.close; + +import org.elasticsearch.Version; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; + +import static org.elasticsearch.test.VersionUtils.randomVersionBetween; +import static org.hamcrest.Matchers.equalTo; + +public class CloseIndexResponseTests extends ESTestCase { + + public void testSerialization() throws Exception { + final CloseIndexResponse response = randomResponse(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + response.writeTo(out); + + final CloseIndexResponse deserializedResponse = new CloseIndexResponse(); + try (StreamInput in = out.bytes().streamInput()) { + deserializedResponse.readFrom(in); + } + assertCloseIndexResponse(deserializedResponse, response); + } + } + + public void testBwcSerialization() throws Exception { + { + final CloseIndexResponse response = randomResponse(); + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.setVersion(randomVersionBetween(random(), Version.V_6_0_0, VersionUtils.getPreviousVersion(Version.V_8_0_0))); + response.writeTo(out); + + final AcknowledgedResponse deserializedResponse = new AcknowledgedResponse(); + try (StreamInput in = out.bytes().streamInput()) { + deserializedResponse.readFrom(in); + } + assertThat(deserializedResponse.isAcknowledged(), equalTo(response.isAcknowledged())); + } + } + { + final AcknowledgedResponse response = new AcknowledgedResponse(randomBoolean()); + try (BytesStreamOutput out = new BytesStreamOutput()) { + response.writeTo(out); + + final CloseIndexResponse deserializedResponse = new CloseIndexResponse(); + try (StreamInput in = out.bytes().streamInput()) { + in.setVersion(randomVersionBetween(random(), Version.V_6_0_0, VersionUtils.getPreviousVersion(Version.V_8_0_0))); + deserializedResponse.readFrom(in); + } + assertThat(deserializedResponse.isAcknowledged(), equalTo(response.isAcknowledged())); + } + } + } + + private CloseIndexResponse randomResponse() { + final boolean acknowledged = randomBoolean(); + final boolean shardsAcknowledged = acknowledged ? randomBoolean() : false; + return new CloseIndexResponse(acknowledged, shardsAcknowledged); + } + + private static void assertCloseIndexResponse(final CloseIndexResponse actual, final CloseIndexResponse expected) { + assertThat(actual.isAcknowledged(), equalTo(expected.isAcknowledged())); + assertThat(actual.isShardsAcknowledged(), equalTo(expected.isShardsAcknowledged())); + } +} 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 d9c7e26f01da4..78395cf544ad3 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -915,7 +915,7 @@ public void testShardChangesWithDefaultDocType() throws Exception { * Test that the {@link org.elasticsearch.index.engine.NoOpEngine} takes precedence over other * engine factories if the index is closed. */ - public void testNoOpEngineFactoryTakesPrecedence() throws IOException { + public void testNoOpEngineFactoryTakesPrecedence() { final String indexName = "closed-index"; createIndex(indexName, Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0).build()); ensureGreen(); @@ -927,7 +927,7 @@ public void testNoOpEngineFactoryTakesPrecedence() throws IOException { final IndexMetaData indexMetaData = clusterState.metaData().index(indexName); final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.createIndex(indexMetaData, Collections.emptyList()); + final IndexService indexService = indicesService.indexServiceSafe(indexMetaData.getIndex()); for (IndexShard indexShard : indexService) { assertThat(indexShard.getEngine(), instanceOf(NoOpEngine.class)); diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java index 81cea988cd020..ac83c50fea6ae 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -211,8 +212,13 @@ public void testIndexStateShardChanged() throws Throwable { assertThat(stateChangeListenerNode1.afterCloseSettings.getAsInt(SETTING_NUMBER_OF_SHARDS, -1), equalTo(6)); assertThat(stateChangeListenerNode1.afterCloseSettings.getAsInt(SETTING_NUMBER_OF_REPLICAS, -1), equalTo(1)); - assertShardStatesMatch(stateChangeListenerNode1, 6, CLOSED); - assertShardStatesMatch(stateChangeListenerNode2, 6, CLOSED); + if (Version.CURRENT.onOrAfter(Version.V_8_0_0)) { + assertShardStatesMatch(stateChangeListenerNode1, 6, CLOSED, CREATED, RECOVERING, POST_RECOVERY, STARTED); + assertShardStatesMatch(stateChangeListenerNode2, 6, CLOSED, CREATED, RECOVERING, POST_RECOVERY, STARTED); + } else { + assertShardStatesMatch(stateChangeListenerNode1, 6, CLOSED); + assertShardStatesMatch(stateChangeListenerNode2, 6, CLOSED); + } } private static void assertShardStatesMatch(final IndexShardStateChangeListener stateChangeListener, diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java index 42f29e99982cc..62d72c3f71954 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java @@ -20,9 +20,11 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.routing.ShardRouting; @@ -112,7 +114,8 @@ public void testCloseAlreadyClosedIndex() throws Exception { assertIndexIsClosed(indexName); // Second close should be acked too - assertBusy(() -> assertAcked(client().admin().indices().prepareClose(indexName))); + final ActiveShardCount activeShardCount = randomFrom(ActiveShardCount.NONE, ActiveShardCount.DEFAULT, ActiveShardCount.ALL); + assertBusy(() -> assertAcked(client().admin().indices().prepareClose(indexName).setWaitForActiveShards(activeShardCount))); assertIndexIsClosed(indexName); } @@ -126,7 +129,7 @@ public void testCloseUnassignedIndex() throws Exception { assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.OPEN)); assertThat(clusterState.routingTable().allShards().stream().allMatch(ShardRouting::unassigned), is(true)); - assertBusy(() -> assertAcked(client().admin().indices().prepareClose(indexName))); + assertBusy(() -> assertAcked(client().admin().indices().prepareClose(indexName).setWaitForActiveShards(ActiveShardCount.NONE))); assertIndexIsClosed(indexName); } @@ -305,6 +308,25 @@ public void testConcurrentClosesAndOpens() throws Exception { indexer.totalIndexedDocs()); } + public void testCloseIndexWaitForActiveShards() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, Settings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0) // no replicas to avoid recoveries that could fail the index closing + .build()); + + final int nbDocs = randomIntBetween(0, 50); + indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, nbDocs) + .mapToObj(i -> client().prepareIndex(indexName, "_doc", String.valueOf(i)).setSource("num", i)).collect(toList())); + ensureGreen(indexName); + + final CloseIndexResponse closeIndexResponse = client().admin().indices().prepareClose(indexName).get(); + assertThat(client().admin().cluster().prepareHealth(indexName).get().getStatus(), is(ClusterHealthStatus.GREEN)); + assertTrue(closeIndexResponse.isAcknowledged()); + assertTrue(closeIndexResponse.isShardsAcknowledged()); + assertIndexIsClosed(indexName); + } + static void assertIndexIsClosed(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); for (String index : indices) { diff --git a/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java index 083c5ab1f5510..25d8f07bbd1cd 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java @@ -20,8 +20,8 @@ package org.elasticsearch.indices.state; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction; -import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -72,7 +72,7 @@ public void testReopenDuringClose() throws Exception { final CountDownLatch block = new CountDownLatch(1); final Releasable releaseBlock = interceptVerifyShardBeforeCloseActions(indexName, block::countDown); - ActionFuture closeIndexResponse = client().admin().indices().prepareClose(indexName).execute(); + ActionFuture closeIndexResponse = client().admin().indices().prepareClose(indexName).execute(); assertTrue("Waiting for index to have a closing blocked", block.await(60, TimeUnit.SECONDS)); assertIndexIsBlocked(indexName); assertFalse(closeIndexResponse.isDone()); @@ -96,7 +96,7 @@ public void testReopenDuringCloseOnMultipleIndices() throws Exception { final CountDownLatch block = new CountDownLatch(1); final Releasable releaseBlock = interceptVerifyShardBeforeCloseActions(randomFrom(indices), block::countDown); - ActionFuture closeIndexResponse = client().admin().indices().prepareClose("index-*").execute(); + ActionFuture closeIndexResponse = client().admin().indices().prepareClose("index-*").execute(); assertTrue("Waiting for index to have a closing blocked", block.await(60, TimeUnit.SECONDS)); assertFalse(closeIndexResponse.isDone()); indices.forEach(ReopenWhileClosingIT::assertIndexIsBlocked); diff --git a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java index 050d77a223101..854dba7fb894b 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java @@ -102,7 +102,7 @@ public void testFastCloseAfterCreateContinuesCreateAfterOpen() { assertThat(health.isTimedOut(), equalTo(false)); assertThat(health.getStatus(), equalTo(ClusterHealthStatus.RED)); - assertAcked(client().admin().indices().prepareClose("test")); + assertAcked(client().admin().indices().prepareClose("test").setWaitForActiveShards(ActiveShardCount.NONE)); logger.info("--> updating test index settings to allow allocation"); client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 46b3c6e54f576..8df54c54e1478 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.admin.indices.open.OpenIndexRequest; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; @@ -193,7 +194,7 @@ private void closeIndexUpdateSettingsAndOpenIndex(String followIndex, Runnable handler, Consumer onFailure) { CloseIndexRequest closeRequest = new CloseIndexRequest(followIndex); - CheckedConsumer onResponse = response -> { + CheckedConsumer onResponse = response -> { updateSettingsAndOpenIndex(followIndex, updatedSettings, handler, onFailure); }; followerClient.admin().indices().close(closeRequest, ActionListener.wrap(onResponse, onFailure)); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java index 2f111727b08ee..7f93934fd91f8 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CloseFollowerIndexIT.java @@ -26,6 +26,7 @@ import java.security.AccessController; import java.security.PrivilegedAction; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonMap; @@ -96,7 +97,10 @@ public void testCloseAndReopenFollowerIndex() throws Exception { } atLeastDocsIndexed(followerClient(), "index2", 32); - AcknowledgedResponse response = followerClient().admin().indices().close(new CloseIndexRequest("index2")).get(); + + CloseIndexRequest closeIndexRequest = new CloseIndexRequest("index2"); + closeIndexRequest.waitForActiveShards(ActiveShardCount.NONE); + AcknowledgedResponse response = followerClient().admin().indices().close(closeIndexRequest).get(); assertThat(response.isAcknowledged(), is(true)); ClusterState clusterState = followerClient().admin().cluster().prepareState().get().getState(); @@ -126,6 +130,6 @@ public void testCloseAndReopenFollowerIndex() throws Exception { followerSearchRequest.source().trackTotalHits(true); long followerIndexDocs = followerClient().search(followerSearchRequest).actionGet().getHits().getTotalHits().value; assertThat(followerIndexDocs, equalTo(leaderIndexDocs)); - }); + }, 30L, TimeUnit.SECONDS); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java index 1efe5389d81b2..91b91ddd04f3c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; @@ -126,9 +127,9 @@ protected void masterOperation(Task task, TransportFreezeIndexAction.FreezeReque .masterNodeTimeout(request.masterNodeTimeout()) .indices(concreteIndices); - indexStateService.closeIndices(closeRequest, new ActionListener() { + indexStateService.closeIndices(closeRequest, new ActionListener() { @Override - public void onResponse(final AcknowledgedResponse response) { + public void onResponse(final CloseIndexResponse response) { if (response.isAcknowledged()) { toggleFrozenSettings(concreteIndices, request, listener); } else { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStepTests.java index 25e1c4e481bba..368afaa26d0cc 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStepTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/CloseFollowerIndexStepTests.java @@ -8,7 +8,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; -import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.admin.indices.close.CloseIndexResponse; import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; import org.elasticsearch.client.IndicesAdminClient; @@ -43,8 +43,8 @@ public void testCloseFollowingIndex() { CloseIndexRequest closeIndexRequest = (CloseIndexRequest) invocation.getArguments()[0]; assertThat(closeIndexRequest.indices()[0], equalTo("follower-index")); @SuppressWarnings("unchecked") - ActionListener listener = (ActionListener) invocation.getArguments()[1]; - listener.onResponse(new AcknowledgedResponse(true)); + ActionListener listener = (ActionListener) invocation.getArguments()[1]; + listener.onResponse(new CloseIndexResponse(true, true)); return null; }).when(indicesClient).close(Mockito.any(), Mockito.any()); From c6c42a14217f0e90eb44c00327899b9f8bcfb8cd Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 28 Feb 2019 10:46:16 +0100 Subject: [PATCH 19/20] Adapt NoOpEngineTests after #39006 --- .../java/org/elasticsearch/index/engine/NoOpEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java index 9a088a85ac7a1..b70ccf03aacaf 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java @@ -126,7 +126,7 @@ public void testNoOpEngineDocStats() throws Exception { deletions += 1; } } - engine.waitForOpsToComplete(numDocs + deletions - 1); + engine.getLocalCheckpointTracker().waitForOpsToComplete(numDocs + deletions - 1); flushAndTrimTranslog(engine); } From 79b9a5ac8efa150aaf9718a920ff397196a79233 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 28 Feb 2019 16:02:41 +0100 Subject: [PATCH 20/20] Amend comment --- .../action/admin/indices/close/CloseIndexRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java index e7979beb68214..10bba4359c4a1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexRequest.java @@ -40,7 +40,7 @@ public class CloseIndexRequest extends AcknowledgedRequest im private String[] indices; private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen(); - private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT; //NORELEASE Changes this to NONE to keep previous behavior + private ActiveShardCount waitForActiveShards = ActiveShardCount.DEFAULT; // Changes this to NONE on 7.x to keep previous behavior public CloseIndexRequest() { }