From ebeb324fb9e625c9d18ff51c8526acedda3ec230 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 17 Apr 2015 22:37:46 +0200 Subject: [PATCH 01/38] support in syncCommitIfNoPendingChanges in IndexShard and Engine. Also added commit id to flush + test. Added a service to allow us to start working.. --- .../elasticsearch/index/engine/Engine.java | 18 ++++++- .../index/engine/InternalEngine.java | 52 ++++++++++++++++--- .../index/engine/ShadowEngine.java | 19 ++++--- .../elasticsearch/index/shard/IndexShard.java | 10 ++-- .../index/shard/IndexShardModule.java | 1 + .../index/shard/SyncCommitService.java | 38 ++++++++++++++ .../index/engine/InternalEngineTests.java | 32 +++++++----- .../index/shard/SyncCommitTests.java | 38 ++++++++++++++ 8 files changed, 174 insertions(+), 34 deletions(-) create mode 100644 src/main/java/org/elasticsearch/index/shard/SyncCommitService.java create mode 100644 src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 392a663d293..c2905dbf1f3 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -66,6 +66,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; */ public abstract class Engine implements Closeable { + public static final String SYNC_COMMIT_ID = "sync_id"; + protected final ShardId shardId; protected final ESLogger logger; protected final EngineConfig engineConfig; @@ -204,6 +206,15 @@ public abstract class Engine implements Closeable { public abstract void delete(DeleteByQuery delete) throws EngineException; + /** + * Attempts to do a special commit where the given syncID is put into the commit data. The attempt + * succeeds if there are not pending writes in lucene and the current point is equal to the expected one. + * @param syncId id of this sync + * @param expectedCommitId the expected value of + * @return true if the sync commit was made, false o.w. + */ + public abstract boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; + final protected GetResult getFromSearcher(Get get) throws EngineException { final Searcher searcher = acquireSearcher("get"); final Versions.DocIdAndVersion docIdAndVersion; @@ -415,16 +426,19 @@ public abstract class Engine implements Closeable { * @param force if true a lucene commit is executed even if no changes need to be committed. * @param waitIfOngoing if true this call will block until all currently running flushes have finished. * Otherwise this call will return without blocking. + * @return the commit Id for the resulting commit */ - public abstract void flush(boolean force, boolean waitIfOngoing) throws EngineException; + public abstract byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException; /** * Flushes the state of the engine including the transaction log, clearing memory and persisting * documents in the lucene index to disk including a potentially heavy and durable fsync operation. * This operation is not going to block if another flush operation is currently running and won't write * a lucene commit if nothing needs to be committed. + * + * @return the commit Id for the resulting commit */ - public abstract void flush() throws EngineException; + public abstract byte[] flush() throws EngineException; /** * Optimizes to 1 segment diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index b4db6b93f17..79938da3a2c 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -103,7 +103,7 @@ public class InternalEngine extends Engine { private final AtomicLong translogIdGenerator = new AtomicLong(); private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean(); - private SegmentInfos lastCommittedSegmentInfos; + private volatile SegmentInfos lastCommittedSegmentInfos; private final IndexThrottle throttle; @@ -597,16 +597,53 @@ public class InternalEngine extends Engine { } @Override - public void flush() throws EngineException { - flush(true, false, false); + public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { + // best effort attempt before we aquire locks + ensureOpen(); + if (indexWriter.hasUncommittedChanges()) { + logger.trace("can't sync commit [{}]. have pending changes", syncId); + return false; + } + if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { + logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); + return false; + } + try (ReleasableLock lock = writeLock.acquire()) { + ensureOpen(); + if (indexWriter.hasUncommittedChanges()) { + logger.trace("can't sync commit [{}]. have pending changes", syncId); + return false; + } + if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { + logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); + return false; + } + logger.trace("starting sync commit [{}]", syncId); + long translogId = translog.currentId(); + Map commitData = new HashMap<>(2); + commitData.put(SYNC_COMMIT_ID, syncId); + commitData.put(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)); + indexWriter.setCommitData(commitData); + commitIndexWriter(indexWriter); + logger.debug("successfully sync committed. sync id [{}].", syncId); + return true; + } catch (IOException ex) { + maybeFailEngine("sync commit", ex); + throw new EngineException(shardId, "failed to sync commit", ex); + } } @Override - public void flush(boolean force, boolean waitIfOngoing) throws EngineException { - flush(true, force, waitIfOngoing); + public byte[] flush() throws EngineException { + return flush(true, false, false); } - private void flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException { + @Override + public byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException { + return flush(true, force, waitIfOngoing); + } + + private byte[] flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException { ensureOpen(); if (commitTranslog) { // check outside the lock as well so we can check without blocking on the write lock @@ -614,6 +651,7 @@ public class InternalEngine extends Engine { throw new FlushNotAllowedEngineException(shardId, "recovery is in progress, flush with committing translog is not allowed"); } } + final byte[] newCommitId; /* * Unfortunately the lock order is important here. We have to acquire the readlock first otherwise * if we are flushing at the end of the recovery while holding the write lock we can deadlock if: @@ -704,6 +742,7 @@ public class InternalEngine extends Engine { } finally { store.decRef(); } + newCommitId = lastCommittedSegmentInfos.getId(); } catch (FlushFailedEngineException ex) { maybeFailEngine("flush", ex); throw ex; @@ -716,6 +755,7 @@ public class InternalEngine extends Engine { if (engineConfig.isEnableGcDeletes()) { pruneDeletedTombstones(); } + return newCommitId; } private void pruneDeletedTombstones() { diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 31c5a23c578..d0e4fce2bd5 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -20,16 +20,12 @@ package org.elasticsearch.index.engine; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SegmentReader; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.util.concurrent.ReleasableLock; @@ -38,9 +34,6 @@ import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; /** * ShadowEngine is a specialized engine that only allows read-only operations @@ -115,12 +108,17 @@ public class ShadowEngine extends Engine { } @Override - public void flush() throws EngineException { - flush(false, false); + public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } @Override - public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + public byte[] flush() throws EngineException { + return flush(false, false); + } + + @Override + public byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.trace("skipping FLUSH on shadow engine"); // reread the last committed segment infos refresh("flush"); @@ -144,6 +142,7 @@ public class ShadowEngine extends Engine { } finally { store.decRef(); } + return lastCommittedSegmentInfos.getId(); } @Override diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 1ce97d60d48..4e1594815e1 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -123,8 +123,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import static org.elasticsearch.index.mapper.SourceToParse.source; - /** * */ @@ -684,6 +682,12 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } + public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + verifyStartedOrRecovering(); + logger.trace("tryimg to sync commit. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); + return engine().syncCommitIfNoPendingChanges(syncId, expectedCommitId); + } + public void flush(FlushRequest request) throws ElasticsearchException { // we allows flush while recovering, since we allow for operations to happen // while recovering, and we want to keep the translog at bay (up to deletes, which @@ -703,7 +707,7 @@ public class IndexShard extends AbstractIndexShardComponent { logger.trace("optimize with {}", optimize); } engine().forceMerge(optimize.flush(), optimize.maxNumSegments(), optimize.onlyExpungeDeletes(), - optimize.upgrade(), optimize.upgradeOnlyAncientSegments()); + optimize.upgrade(), optimize.upgradeOnlyAncientSegments()); } public SnapshotIndexCommit snapshotIndex() throws EngineException { diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index 2ba09533eae..ef3889c4a8f 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -65,6 +65,7 @@ public class IndexShardModule extends AbstractModule { bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX)); bind(ShardIndexWarmerService.class).asEagerSingleton(); + bind(SyncCommitService.class).asEagerSingleton(); } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/index/shard/SyncCommitService.java b/src/main/java/org/elasticsearch/index/shard/SyncCommitService.java new file mode 100644 index 00000000000..3eae4978f71 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/shard/SyncCommitService.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.index.shard; + +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.settings.IndexSettings; + +public class SyncCommitService extends AbstractIndexShardComponent { + + private final IndexShard indexShard; + + @Inject + public SyncCommitService(ShardId shardId, @IndexSettings Settings indexSettings, IndexShard indexShard) { + super(shardId, indexSettings); + this.indexShard = indexShard; + } + + public boolean attemptSyncCommit() { + throw new UnsupportedOperationException("not so fast"); + } +} diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index ad9d9c57cec..b0d012eac14 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -28,12 +28,7 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexDeletionPolicy; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LiveIndexWriterConfig; -import org.apache.lucene.index.Term; +import org.apache.lucene.index.*; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; @@ -102,18 +97,13 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean; -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomDouble; -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; +import static com.carrotsearch.randomizedtesting.RandomizedTest.*; import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.test.ElasticsearchTestCase.assertBusy; import static org.elasticsearch.test.ElasticsearchTestCase.terminate; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; @LuceneTestCase.SuppressFileSystems("*") // mock FS causes translog issues recovering sometimes because of their use of globs, see LUCENE-6424 public class InternalEngineTests extends ElasticsearchLuceneTestCase { @@ -681,6 +671,22 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase { searchResult.close(); } + public void testSyncCommit() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + byte[] commitID = engine.flush(); + assertThat(commitID, equalTo(store.readLastCommittedSegmentsInfo().getId())); + byte[] fakeId = commitID.clone(); + fakeId[0] = (byte) ~fakeId[0]; + assertFalse("should fail to sync commit with wrong id (but no docs)", engine.syncCommitIfNoPendingChanges(syncId + "1", fakeId)); + engine.create(new Engine.Create(null, newUid("2"), doc)); + assertFalse("should fail to sync commit with right id but pending doc", engine.syncCommitIfNoPendingChanges(syncId + "2", commitID)); + commitID = engine.flush(); + assertTrue("should succeed to sync commit with right id and no pending doc", engine.syncCommitIfNoPendingChanges(syncId, commitID)); + assertThat(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); + } + @Test public void testFailEngineOnCorruption() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); diff --git a/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java b/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java new file mode 100644 index 00000000000..cd9f6b351b3 --- /dev/null +++ b/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java @@ -0,0 +1,38 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.index.shard; + +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.test.ElasticsearchIntegrationTest; + +public class SyncCommitTests extends ElasticsearchIntegrationTest { + + public void testSyncCommit() { + internalCluster().ensureAtLeastNumDataNodes(2); + prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get(); + ensureGreen(); + ClusterStateResponse state = client().admin().cluster().prepareState().get(); + String nodeId = state.getState().getRoutingTable().index("test").shard(0).getShards().get(0).currentNodeId(); + String nodeName = state.getState().getNodes().get(nodeId).name(); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, nodeName); + indicesService.indexServiceSafe("test").shardInjectorSafe(0).getInstance(SyncCommitService.class).attemptSyncCommit(); + } +} From 0bad8a771454ec389ee8d5eb8021839d77caaba8 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 18 Apr 2015 13:19:42 +0200 Subject: [PATCH 02/38] moved to synced flush terminology plus a node level service for now --- .../elasticsearch/index/engine/Engine.java | 2 +- .../index/engine/InternalEngine.java | 2 +- .../index/engine/ShadowEngine.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 6 +-- .../index/shard/IndexShardModule.java | 1 - .../elasticsearch/indices/IndicesModule.java | 2 +- .../SyncedFlushService.java} | 17 +++++---- .../index/engine/InternalEngineTests.java | 8 ++-- .../index/shard/SyncCommitTests.java | 38 ------------------- .../org/elasticsearch/indices/FlushTest.java | 18 +++++++-- 10 files changed, 34 insertions(+), 62 deletions(-) rename src/main/java/org/elasticsearch/{index/shard/SyncCommitService.java => indices/SyncedFlushService.java} (68%) delete mode 100644 src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index c2905dbf1f3..1c978af0d5c 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -213,7 +213,7 @@ public abstract class Engine implements Closeable { * @param expectedCommitId the expected value of * @return true if the sync commit was made, false o.w. */ - public abstract boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; + public abstract boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; final protected GetResult getFromSearcher(Get get) throws EngineException { final Searcher searcher = acquireSearcher("get"); diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 79938da3a2c..ec7f72965eb 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -597,7 +597,7 @@ public class InternalEngine extends Engine { } @Override - public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { + public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { // best effort attempt before we aquire locks ensureOpen(); if (indexWriter.hasUncommittedChanges()) { diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index d0e4fce2bd5..5197d4da9df 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -108,7 +108,7 @@ public class ShadowEngine extends Engine { } @Override - public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4e1594815e1..43216888d38 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -682,10 +682,10 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } - public boolean syncCommitIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { verifyStartedOrRecovering(); - logger.trace("tryimg to sync commit. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); - return engine().syncCommitIfNoPendingChanges(syncId, expectedCommitId); + logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); + return engine().syncFlushIfNoPendingChanges(syncId, expectedCommitId); } public void flush(FlushRequest request) throws ElasticsearchException { diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index ef3889c4a8f..2ba09533eae 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -65,7 +65,6 @@ public class IndexShardModule extends AbstractModule { bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX)); bind(ShardIndexWarmerService.class).asEagerSingleton(); - bind(SyncCommitService.class).asEagerSingleton(); } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/indices/IndicesModule.java b/src/main/java/org/elasticsearch/indices/IndicesModule.java index 102fa1854d4..785d2af7eca 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -20,7 +20,6 @@ package org.elasticsearch.indices; import com.google.common.collect.ImmutableList; - import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.Module; @@ -69,6 +68,7 @@ public class IndicesModule extends AbstractModule implements SpawnModules { bind(IndicesStore.class).asEagerSingleton(); bind(IndicesClusterStateService.class).asEagerSingleton(); bind(IndexingMemoryController.class).asEagerSingleton(); + bind(SyncedFlushService.class).asEagerSingleton(); bind(IndicesFilterCache.class).asEagerSingleton(); bind(IndicesQueryCache.class).asEagerSingleton(); bind(IndicesFieldDataCache.class).asEagerSingleton(); diff --git a/src/main/java/org/elasticsearch/index/shard/SyncCommitService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java similarity index 68% rename from src/main/java/org/elasticsearch/index/shard/SyncCommitService.java rename to src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 3eae4978f71..01990c802ee 100644 --- a/src/main/java/org/elasticsearch/index/shard/SyncCommitService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -16,23 +16,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.index.shard; +package org.elasticsearch.indices; +import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.settings.IndexSettings; +import org.elasticsearch.index.shard.ShardId; -public class SyncCommitService extends AbstractIndexShardComponent { +public class SyncedFlushService extends AbstractComponent { - private final IndexShard indexShard; + private final IndicesService indicesService; @Inject - public SyncCommitService(ShardId shardId, @IndexSettings Settings indexSettings, IndexShard indexShard) { - super(shardId, indexSettings); - this.indexShard = indexShard; + public SyncedFlushService(Settings settings, IndicesService indicesService) { + super(settings); + this.indicesService = indicesService; } - public boolean attemptSyncCommit() { + public boolean attemptSyncedFlush(ShardId shardId) { throw new UnsupportedOperationException("not so fast"); } } diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index b0d012eac14..dbf2c106f41 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -671,7 +671,7 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase { searchResult.close(); } - public void testSyncCommit() throws IOException { + public void testSyncedFlush() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); engine.create(new Engine.Create(null, newUid("1"), doc)); @@ -679,11 +679,11 @@ public class InternalEngineTests extends ElasticsearchLuceneTestCase { assertThat(commitID, equalTo(store.readLastCommittedSegmentsInfo().getId())); byte[] fakeId = commitID.clone(); fakeId[0] = (byte) ~fakeId[0]; - assertFalse("should fail to sync commit with wrong id (but no docs)", engine.syncCommitIfNoPendingChanges(syncId + "1", fakeId)); + assertFalse("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", fakeId)); engine.create(new Engine.Create(null, newUid("2"), doc)); - assertFalse("should fail to sync commit with right id but pending doc", engine.syncCommitIfNoPendingChanges(syncId + "2", commitID)); + assertFalse("should fail to sync flush with right id but pending doc", engine.syncFlushIfNoPendingChanges(syncId + "2", commitID)); commitID = engine.flush(); - assertTrue("should succeed to sync commit with right id and no pending doc", engine.syncCommitIfNoPendingChanges(syncId, commitID)); + assertTrue("should succeed to flush commit with right id and no pending doc", engine.syncFlushIfNoPendingChanges(syncId, commitID)); assertThat(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); } diff --git a/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java b/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java deleted file mode 100644 index cd9f6b351b3..00000000000 --- a/src/test/java/org/elasticsearch/index/shard/SyncCommitTests.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.index.shard; - -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.test.ElasticsearchIntegrationTest; - -public class SyncCommitTests extends ElasticsearchIntegrationTest { - - public void testSyncCommit() { - internalCluster().ensureAtLeastNumDataNodes(2); - prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get(); - ensureGreen(); - ClusterStateResponse state = client().admin().cluster().prepareState().get(); - String nodeId = state.getState().getRoutingTable().index("test").shard(0).getShards().get(0).currentNodeId(); - String nodeName = state.getState().getNodes().get(nodeId).name(); - IndicesService indicesService = internalCluster().getInstance(IndicesService.class, nodeName); - indicesService.indexServiceSafe("test").shardInjectorSafe(0).getInstance(SyncCommitService.class).attemptSyncCommit(); - } -} diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index 161ad6c0553..77ca89a4715 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -19,18 +19,17 @@ package org.elasticsearch.indices; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.flush.FlushResponse; -import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; import java.util.Arrays; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; @@ -72,4 +71,15 @@ public class FlushTest extends ElasticsearchIntegrationTest { assertThat(errors, emptyIterable()); } } + + public void testSyncedFlush() { + internalCluster().ensureAtLeastNumDataNodes(2); + prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get(); + ensureGreen(); + ClusterStateResponse state = client().admin().cluster().prepareState().get(); + String nodeId = state.getState().getRoutingTable().index("test").shard(0).getShards().get(0).currentNodeId(); + String nodeName = state.getState().getNodes().get(nodeId).name(); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, nodeName); + indicesService.indexServiceSafe("test").shardInjectorSafe(0).getInstance(SyncedFlushService.class).attemptSyncedFlush(new ShardId("test", 0)); + } } From 3633b83b1f533332b13c67062115424bf1180f9a Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Wed, 22 Apr 2015 14:16:08 +0200 Subject: [PATCH 03/38] sync commit actions --- .../action/bulk/TransportShardBulkAction.java | 11 +- .../action/delete/TransportDeleteAction.java | 11 +- .../TransportDeleteByQueryAction.java | 3 +- .../TransportIndexDeleteByQueryAction.java | 3 +- .../TransportShardDeleteByQueryAction.java | 11 +- .../action/index/TransportIndexAction.java | 15 +- .../TransportBroadcastOperationAction.java | 7 +- ...nsportIndexReplicationOperationAction.java | 7 +- ...portIndicesReplicationOperationAction.java | 7 +- ...nsportShardReplicationOperationAction.java | 115 ++++++++++----- .../client/IndicesAdminClient.java | 2 +- .../cluster/routing/RoutingTable.java | 32 ++++- .../common/io/stream/StreamInput.java | 23 ++- .../common/io/stream/StreamOutput.java | 10 ++ .../index/engine/InternalEngine.java | 3 +- .../elasticsearch/index/shard/IndexShard.java | 15 +- .../elasticsearch/indices/IndicesModule.java | 1 + .../indices/SyncedFlushService.java | 39 ----- .../syncedflush/PreSyncedFlushRequest.java | 70 +++++++++ .../syncedflush/PreSyncedFlushResponse.java | 73 ++++++++++ .../PreSyncedShardFlushRequest.java | 72 ++++++++++ .../PreSyncedShardFlushResponse.java | 67 +++++++++ .../SyncedFlushReplicaResponse.java | 80 +++++++++++ .../syncedflush/SyncedFlushRequest.java | 89 ++++++++++++ .../syncedflush/SyncedFlushResponse.java | 66 +++++++++ .../syncedflush/SyncedFlushService.java | 48 +++++++ .../TransportPreSyncedFlushAction.java | 135 +++++++++++++++++ .../TransportSyncedFlushAction.java | 136 ++++++++++++++++++ .../cluster/routing/RoutingTableTest.java | 99 +++++++------ .../index/engine/InternalEngineTests.java | 1 + .../org/elasticsearch/indices/FlushTest.java | 29 +++- .../SynceFlushStreamablesTests.java | 93 ++++++++++++ 32 files changed, 1217 insertions(+), 156 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/indices/SyncedFlushService.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushRequest.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java create mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java create mode 100644 src/test/java/org/elasticsearch/indices/syncedflush/SynceFlushStreamablesTests.java diff --git a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 245d7d16033..16455b0d1be 100644 --- a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -60,6 +60,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.river.RiverIndexName; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -67,7 +68,7 @@ import java.util.Map; /** * Performs the index operation. */ -public class TransportShardBulkAction extends TransportShardReplicationOperationAction { +public class TransportShardBulkAction extends TransportShardReplicationOperationAction { private final static String OP_TYPE_UPDATE = "update"; private final static String OP_TYPE_DELETE = "delete"; @@ -118,6 +119,11 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation return new BulkShardResponse(); } + @Override + protected TransportResponse.Empty newReplicaResponseInstance() { + return TransportResponse.Empty.INSTANCE; + } + @Override protected boolean resolveIndex() { return false; @@ -528,7 +534,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation @Override - protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); final BulkShardRequest request = shardRequest.request; for (int i = 0; i < request.items().length; i++) { @@ -580,6 +586,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation // ignore } } + return newReplicaResponseInstance(); } private void applyVersion(BulkItemRequest item, long version, VersionType versionType) { diff --git a/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 86ffac8bcce..efdea2cb3b5 100644 --- a/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -43,12 +43,13 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** * Performs the delete operation. */ -public class TransportDeleteAction extends TransportShardReplicationOperationAction { +public class TransportDeleteAction extends TransportShardReplicationOperationAction { private final AutoCreateIndex autoCreateIndex; @@ -140,6 +141,11 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct return new DeleteResponse(); } + @Override + protected TransportResponse.Empty newReplicaResponseInstance() { + return TransportResponse.Empty.INSTANCE; + } + @Override protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { DeleteRequest request = shardRequest.request; @@ -165,7 +171,7 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct } @Override - protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { DeleteRequest request = shardRequest.request; IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA); @@ -179,6 +185,7 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct // ignore } } + return newReplicaResponseInstance(); } @Override diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java index 0800a639a81..2571eecef5f 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -40,7 +41,7 @@ import java.util.concurrent.atomic.AtomicReferenceArray; /** */ -public class TransportDeleteByQueryAction extends TransportIndicesReplicationOperationAction { +public class TransportDeleteByQueryAction extends TransportIndicesReplicationOperationAction { private final DestructiveOperations destructiveOperations; diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java index 607459e7798..a76a0ee9e23 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java @@ -30,13 +30,14 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import java.util.List; /** * Internal transport action that broadcasts a delete by query request to all of the shards that belong to an index. */ -public class TransportIndexDeleteByQueryAction extends TransportIndexReplicationOperationAction { +public class TransportIndexDeleteByQueryAction extends TransportIndexReplicationOperationAction { private static final String ACTION_NAME = DeleteByQueryAction.NAME + "[index]"; diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java index 6e364302e83..aeb852e967e 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java @@ -42,12 +42,13 @@ import org.elasticsearch.search.internal.DefaultSearchContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchLocalRequest; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** * */ -public class TransportShardDeleteByQueryAction extends TransportShardReplicationOperationAction { +public class TransportShardDeleteByQueryAction extends TransportShardReplicationOperationAction { public final static String DELETE_BY_QUERY_API = "delete_by_query"; @@ -93,6 +94,11 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication return new ShardDeleteByQueryResponse(); } + @Override + protected TransportResponse.Empty newReplicaResponseInstance() { + return TransportResponse.Empty.INSTANCE; + } + @Override protected boolean resolveIndex() { return false; @@ -121,7 +127,7 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication @Override - protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { ShardDeleteByQueryRequest request = shardRequest.request; IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); @@ -138,6 +144,7 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication SearchContext.removeCurrent(); } } + return newReplicaResponseInstance(); } @Override diff --git a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 79ea496c317..e165253f0b0 100644 --- a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -49,6 +49,7 @@ import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.river.RiverIndexName; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** @@ -61,7 +62,7 @@ import org.elasticsearch.transport.TransportService; *
  • allowIdGeneration: If the id is set not, should it be generated. Defaults to true. * */ -public class TransportIndexAction extends TransportShardReplicationOperationAction { +public class TransportIndexAction extends TransportShardReplicationOperationAction { private final AutoCreateIndex autoCreateIndex; @@ -156,6 +157,11 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi return new IndexResponse(); } + @Override + protected TransportResponse.Empty newReplicaResponseInstance() { + return TransportResponse.Empty.INSTANCE; + } + @Override protected String executor() { return ThreadPool.Names.INDEX; @@ -239,7 +245,7 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi } @Override - protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); IndexRequest request = shardRequest.request; SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).type(request.type()).id(request.id()) @@ -259,5 +265,10 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi // ignore } } + return newReplicaResponseInstance(); + } + + public String getReplicaActionName() { + return IndexAction.NAME + "[r]"; } } diff --git a/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java b/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java index 09d7bd55447..81676226f90 100644 --- a/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; @@ -60,12 +59,16 @@ public abstract class TransportBroadcastOperationAction listener) { new AsyncBroadcastAction(request, listener).start(); diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java index 53c4984bfc8..c4dddc7e361 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportResponse; import java.util.ArrayList; import java.util.Arrays; @@ -48,15 +49,15 @@ import java.util.concurrent.atomic.AtomicReferenceArray; * It relies on a shard sub-action that gets sent over the transport and executed on each of the shard. * The index provided with the request is expected to be a concrete index, properly resolved by the callers (parent actions). */ -public abstract class TransportIndexReplicationOperationAction +public abstract class TransportIndexReplicationOperationAction extends TransportAction { protected final ClusterService clusterService; - protected final TransportShardReplicationOperationAction shardAction; + protected final TransportShardReplicationOperationAction shardAction; protected TransportIndexReplicationOperationAction(Settings settings, String actionName, ClusterService clusterService, - ThreadPool threadPool, TransportShardReplicationOperationAction shardAction, ActionFilters actionFilters) { + ThreadPool threadPool, TransportShardReplicationOperationAction shardAction, ActionFilters actionFilters) { super(settings, actionName, threadPool, actionFilters); this.clusterService = clusterService; this.shardAction = shardAction; diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java index e2a811202c1..c53b9bbb65d 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BaseTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -42,15 +43,15 @@ import java.util.concurrent.atomic.AtomicReferenceArray; /** */ public abstract class TransportIndicesReplicationOperationAction + ShardRequest extends ShardReplicationOperationRequest, ShardResponse extends ActionWriteResponse, ReplicaResponse extends TransportResponse> extends TransportAction { protected final ClusterService clusterService; - protected final TransportIndexReplicationOperationAction indexAction; + protected final TransportIndexReplicationOperationAction indexAction; protected TransportIndicesReplicationOperationAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, - TransportIndexReplicationOperationAction indexAction, ActionFilters actionFilters) { + TransportIndexReplicationOperationAction indexAction, ActionFilters actionFilters) { super(settings, actionName, threadPool, actionFilters); this.clusterService = clusterService; this.indexAction = indexAction; diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java index df99d045177..244d624cb15 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java @@ -59,12 +59,14 @@ import org.elasticsearch.transport.*; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; /** */ -public abstract class TransportShardReplicationOperationAction extends TransportAction { +public abstract class TransportShardReplicationOperationAction extends TransportAction { protected final TransportService transportService; protected final ClusterService clusterService; @@ -86,18 +88,22 @@ public abstract class TransportShardReplicationOperationAction listener) { new AsyncShardOperationAction(request, listener).start(); @@ -109,15 +115,17 @@ public abstract class TransportShardReplicationOperationAction shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable; + protected abstract Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable; - protected abstract void shardOperationOnReplica(ReplicaOperationRequest shardRequest); + protected abstract ReplicaResponse shardOperationOnReplica(ReplicaOperationRequest shardRequest); protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException; @@ -216,7 +224,7 @@ public abstract class TransportShardReplicationOperationAction { + private class ReplicaOperationTransportHandler extends BaseTransportRequestHandler { @Override public ReplicaOperationRequest newInstance() { @@ -236,13 +244,14 @@ public abstract class TransportShardReplicationOperationAction() { + @Override + public ReplicaResponse newInstance() { + return newReplicaResponseInstance(); + } - @Override - public void handleException(TransportException exp) { - state.onReplicaFailure(nodeId, exp); - logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request()); - if (!ignoreReplicaException(exp)) { - logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp); - shardStateAction.shardFailed(shard, indexMetaData.getUUID(), - "Failed to perform [" + actionName + "] on replica, message [" + ExceptionsHelper.detailedMessage(exp) + "]"); - } - } + @Override + public void handleResponse(ReplicaResponse vResponse) { + state.onReplicaSuccess(vResponse); + } - }); + @Override + public void handleException(TransportException exp) { + state.onReplicaFailure(nodeId, exp); + logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request()); + if (!ignoreReplicaException(exp)) { + logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp); + shardStateAction.shardFailed(shard, indexMetaData.getUUID(), + "Failed to perform [" + actionName + "] on replica, message [" + ExceptionsHelper.detailedMessage(exp) + "]"); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + }); } else { if (internalRequest.request().operationThreaded()) { try { @@ -655,8 +687,8 @@ public abstract class TransportShardReplicationOperationAction 2) { // only for more than 2 in the number of shardIt it makes sense, otherwise its 1 shard with 1 replica, quorum is 1 (which is what it is initialized to) requiredNumber = (shardRoutingTable.getSize() / 2) + 1; @@ -738,7 +770,7 @@ public abstract class TransportShardReplicationOperationAction shardReplicaFailures = ConcurrentCollections.newConcurrentMap(); + // nocommit the Broadcast operations use AtomicReferencArray, Boaz wants to figure out why, this here is just a hack + private final CopyOnWriteArrayList replicaResponses = new CopyOnWriteArrayList<>(); private final AtomicInteger pending; private final int numberOfShardInstances; @@ -800,8 +834,9 @@ public abstract class TransportShardReplicationOperationAction replicaResponses) { + return finalResponse; + } + /** * Internal request class that gets built on each node. Holds the original request plus additional info. */ diff --git a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index 0852643444a..408bbd29641 100644 --- a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -352,7 +352,7 @@ public interface IndicesAdminClient extends ElasticsearchClient listener); + void flush(FlushRequest request, ActionListener listener); /** * Explicitly flush one or more indices (releasing memory from the node). diff --git a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 9f1b5db6c6b..b6666813ce7 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.routing; import com.carrotsearch.hppc.IntSet; import com.google.common.collect.*; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -195,7 +196,7 @@ public class RoutingTable implements Iterable { /** * Return GroupShardsIterator where each assigned shard routing has it's own shard iterator. * - * @param includeEmpty if true, a shard iterator will be added for non-assigned shards as well + * @param includeEmpty if true, a shard iterator will be added for non-assigned shards as well * @param includeRelocationTargets if true, an extra shard iterator will be added for relocating shards. The extra * iterator contains a single ShardRouting pointing at the relocating target */ @@ -254,6 +255,35 @@ public class RoutingTable implements Iterable { return new GroupShardsIterator(set); } + /** + * All the shard copies for the provided shard id grouped. Each group is a single element, consisting + * either of the primary shard of one replica. + * + * @param shardId the shard id for the copies we want + * @return All the shard copies (primary and replicas) for the shardId + * @throws IndexMissingException If an index passed does not exists + * @see IndexRoutingTable#groupByAllIt() + */ + public GroupShardsIterator allActiveShardCopiesGrouped(ShardId shardId) throws IndexMissingException { + // use list here since we need to maintain identity across shards + ArrayList set = new ArrayList<>(); + IndexRoutingTable indexRoutingTable = index(shardId.index().name()); + if (indexRoutingTable == null) { + throw new IndexMissingException(new Index(shardId.index().name())); + } + IndexShardRoutingTable copiesRoutingTable = indexRoutingTable.shard(shardId.id()); + if (copiesRoutingTable != null) { + for (ShardRouting shardRouting : copiesRoutingTable) { + if (shardRouting.active()) { + set.add(shardRouting.shardsIt()); + } + } + } else { + throw new ElasticsearchIllegalStateException(shardId + " does not exist"); + } + return new GroupShardsIterator(set); + } + public static Builder builder() { return new Builder(); } diff --git a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index b05d463c58d..ac0e9b3e2d8 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -419,39 +419,48 @@ public abstract class StreamInput extends InputStream { public int[] readIntArray() throws IOException { int length = readVInt(); int[] values = new int[length]; - for(int i=0; i { + private ShardId shardId; + + + PreSyncedFlushRequest() { + } + + public PreSyncedFlushRequest(ShardId shardId) { + super(Arrays.asList(shardId.getIndex()).toArray(new String[0])); + this.shardId = shardId; + } + + @Override + public String toString() { + return "PreSyncedFlushRequest{" + + "shardId=" + shardId + + '}'; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.shardId = ShardId.readShardId(in); + } + + public ShardId shardId() { + return shardId; + } + + public void shardId(ShardId shardId) { + this.shardId = shardId; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java new file mode 100644 index 00000000000..c757f2f669a --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java @@ -0,0 +1,73 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReferenceArray; + +/** + * A response to pre synced flush action. + */ +public class PreSyncedFlushResponse extends BroadcastOperationResponse { + + Map commitIds = new HashMap<>(); + + PreSyncedFlushResponse() { + } + + public PreSyncedFlushResponse(int totalShards, int successfulShards, int failedShards, List shardFailures, AtomicReferenceArray shardsResponses) { + super(totalShards, successfulShards, failedShards, shardFailures); + for (int i = 0; i < shardsResponses.length(); i++) { + PreSyncedShardFlushResponse preSyncedShardFlushResponse = (PreSyncedShardFlushResponse) shardsResponses.get(i); + commitIds.put(preSyncedShardFlushResponse.shardRouting().currentNodeId(), preSyncedShardFlushResponse.id()); + } + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + int numCommitIds = in.readVInt(); + for (int i = 0; i < numCommitIds; i++) { + commitIds.put(in.readString(), in.readByteArray()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(commitIds.size()); + for (Map.Entry entry : commitIds.entrySet()) { + out.writeString(entry.getKey()); + out.writeByteArray(entry.getValue()); + } + } + + public Map commitIds() { + return commitIds; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java new file mode 100644 index 00000000000..4fa3cbd3c33 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java @@ -0,0 +1,72 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest; +import org.elasticsearch.cluster.routing.ImmutableShardRouting; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * + */ +class PreSyncedShardFlushRequest extends BroadcastShardOperationRequest { + + private ShardRouting shardRouting; + // we need our own request because it has to include the shard routing + private PreSyncedFlushRequest request = new PreSyncedFlushRequest(); + + PreSyncedShardFlushRequest() { + } + + PreSyncedShardFlushRequest(ShardRouting shardRouting, PreSyncedFlushRequest request) { + super(shardRouting.shardId(), request); + this.request = request; + this.shardRouting = shardRouting; + } + + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + request.readFrom(in); + shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); + + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + request.writeTo(out); + shardRouting.writeTo(out); + + } + + PreSyncedFlushRequest getRequest() { + return request; + } + + public ShardRouting shardRouting() { + return shardRouting; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java new file mode 100644 index 00000000000..55f166f3a18 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java @@ -0,0 +1,67 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse; +import org.elasticsearch.cluster.routing.ImmutableShardRouting; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * + */ +class PreSyncedShardFlushResponse extends BroadcastShardOperationResponse { + byte[] id; + private ShardRouting shardRouting; + + PreSyncedShardFlushResponse() { + } + + PreSyncedShardFlushResponse(byte[] id, ShardRouting shardRouting) { + super(shardRouting.shardId()); + this.id = id; + this.shardRouting = shardRouting; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + id = in.readByteArray(); + shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeByteArray(id); + shardRouting.writeTo(out); + } + + byte[] id() { + return id; + } + + public ShardRouting shardRouting() { + return shardRouting; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java new file mode 100644 index 00000000000..a08edb635dc --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java @@ -0,0 +1,80 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.TransportResponse; + +import java.io.IOException; + +public class SyncedFlushReplicaResponse extends TransportResponse { + boolean succeeded = true; + private String index; + private int shardId; + private String nodeId; + private String reason; + + void setResult(boolean succeeded, String index, int shardId, String nodeId, String reason) { + this.succeeded = succeeded; + this.index = index; + this.shardId = shardId; + this.nodeId = nodeId; + this.reason = reason; + } + + public String getIndex() { + return index; + } + + public int getShardId() { + return shardId; + } + + public String getNodeId() { + return nodeId; + } + + public String getReason() { + return reason; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + succeeded = in.readBoolean(); + this.index = in.readString(); + this.shardId = in.readInt(); + this.nodeId = in.readString(); + this.reason = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(succeeded); + out.writeString(index); + out.writeInt(shardId); + out.writeString(nodeId); + out.writeString(reason); + } + +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java new file mode 100644 index 00000000000..10ba56d0f65 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java @@ -0,0 +1,89 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.indices.syncedflush; + +import org.elasticsearch.action.support.replication.ShardReplicationOperationRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.shard.ShardId; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + */ +public class SyncedFlushRequest extends ShardReplicationOperationRequest { + + private String syncId; + private Map commitIds; + private ShardId shardId; + + public SyncedFlushRequest() { + } + + public SyncedFlushRequest(ShardId shardId, String syncId, Map commitIds) { + this.commitIds = commitIds; + this.shardId = shardId; + this.syncId = syncId; + this.index(shardId.index().getName()); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + commitIds = new HashMap<>(); + int numCommitIds = in.readVInt(); + for (int i = 0; i < numCommitIds; i++) { + commitIds.put(in.readString(), in.readByteArray()); + } + syncId = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + out.writeVInt(commitIds.size()); + for (Map.Entry entry : commitIds.entrySet()) { + out.writeString(entry.getKey()); + out.writeByteArray(entry.getValue()); + } + out.writeString(syncId); + } + + @Override + public String toString() { + return "write sync commit {" + shardId + "}"; + } + + public ShardId shardId() { + return shardId; + } + + public String syncId() { + return syncId; + } + + public Map commitIds() { + return commitIds; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java new file mode 100644 index 00000000000..199c7d14978 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java @@ -0,0 +1,66 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + */ +public class SyncedFlushResponse extends ActionWriteResponse { + + + private boolean succes; + + String syncCommitId; + + public SyncedFlushResponse() { + + } + + public SyncedFlushResponse(boolean success, String syncCommitId) { + this.succes = success; + this.syncCommitId = syncCommitId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.succes = in.readBoolean(); + syncCommitId = in.readOptionalString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(succes); + out.writeOptionalString(syncCommitId); + } + + public boolean success() { + return succes; + } + public String getSyncId() { + return syncCommitId; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java new file mode 100644 index 00000000000..5920ef7cf95 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java @@ -0,0 +1,48 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.ShardId; + +import java.util.concurrent.ExecutionException; + +public class SyncedFlushService extends AbstractComponent { + + private final TransportPreSyncedFlushAction transportPreSyncedFlushAction; + private final TransportSyncedFlushAction transportSyncedFlushAction; + + @Inject + public SyncedFlushService(Settings settings, TransportPreSyncedFlushAction transportPreSyncedFlushAction, TransportSyncedFlushAction transportSyncedFlushAction) { + super(settings); + this.transportPreSyncedFlushAction = transportPreSyncedFlushAction; + this.transportSyncedFlushAction = transportSyncedFlushAction; + } + + public SyncedFlushResponse attemptSyncedFlush(ShardId shardId) throws ExecutionException, InterruptedException { + PreSyncedFlushResponse preSyncedFlushResponse = transportPreSyncedFlushAction.execute(new PreSyncedFlushRequest(shardId)).get(); + // exit if this did not work + String syncId = Strings.base64UUID(); + SyncedFlushResponse syncedFlushResponse = transportSyncedFlushAction.execute(new SyncedFlushRequest(shardId, syncId, preSyncedFlushResponse.commitIds())).get(); + return syncedFlushResponse; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java b/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java new file mode 100644 index 00000000000..7a76136fb95 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java @@ -0,0 +1,135 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.indices.syncedflush; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.DefaultShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.List; +import java.util.concurrent.atomic.AtomicReferenceArray; + +import static com.google.common.collect.Lists.newArrayList; + + +/** + * Sync Commit Action. + */ +public class TransportPreSyncedFlushAction extends TransportBroadcastOperationAction { + + private final IndicesService indicesService; + + public static final String NAME = "indices:admin/presyncedflush"; + + @Inject + public TransportPreSyncedFlushAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService, ActionFilters actionFilters) { + super(settings, NAME, threadPool, clusterService, transportService, actionFilters); + this.indicesService = indicesService; + } + + @Override + protected String executor() { + return ThreadPool.Names.FLUSH; + } + + @Override + protected PreSyncedFlushRequest newRequestInstance() { + return new PreSyncedFlushRequest(); + } + + @Override + protected PreSyncedFlushResponse newResponse(PreSyncedFlushRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { + int successfulShards = 0; + int failedShards = 0; + List shardFailures = null; + for (int i = 0; i < shardsResponses.length(); i++) { + Object shardResponse = shardsResponses.get(i); + if (shardResponse == null) { + // a non active shard, ignore + } else if (shardResponse instanceof BroadcastShardOperationFailedException) { + failedShards++; + if (shardFailures == null) { + shardFailures = newArrayList(); + } + shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse)); + } else { + successfulShards++; + } + } + return new PreSyncedFlushResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, shardsResponses); + } + + @Override + protected PreSyncedShardFlushRequest newShardRequest() { + return new PreSyncedShardFlushRequest(); + } + + @Override + protected PreSyncedShardFlushRequest newShardRequest(int numShards, ShardRouting shard, PreSyncedFlushRequest request) { + return new PreSyncedShardFlushRequest(shard, request); + } + + @Override + protected PreSyncedShardFlushResponse newShardResponse() { + return new PreSyncedShardFlushResponse(); + } + + @Override + protected PreSyncedShardFlushResponse shardOperation(PreSyncedShardFlushRequest request) throws ElasticsearchException { + IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); + FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); + byte[] id = indexShard.flush(flushRequest); + return new PreSyncedShardFlushResponse(id, request.shardRouting()); + } + + /** + * The sync commit request works against one primary and all of its copies. + */ + @Override + protected GroupShardsIterator shards(ClusterState clusterState, PreSyncedFlushRequest request, String[] concreteIndices) { + return clusterState.routingTable().allActiveShardCopiesGrouped(request.shardId()); + } + + @Override + protected ClusterBlockException checkGlobalBlock(ClusterState state, PreSyncedFlushRequest request) { + return null; + } + + @Override + protected ClusterBlockException checkRequestBlock(ClusterState state, PreSyncedFlushRequest countRequest, String[] concreteIndices) { + return null; + } +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java b/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java new file mode 100644 index 00000000000..b8213a13226 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java @@ -0,0 +1,136 @@ +/* + * 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.syncedflush; + +import org.elasticsearch.ElasticsearchIllegalStateException; +import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.replication.TransportShardReplicationOperationAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; + +/** + */ +public class TransportSyncedFlushAction extends TransportShardReplicationOperationAction { + + public static final String NAME = "indices:admin/syncedflush"; + + @Inject + public TransportSyncedFlushAction(Settings settings, TransportService transportService, ClusterService clusterService, + IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, + ActionFilters actionFilters) { + super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters); + } + + @Override + protected boolean checkWriteConsistency() { + return true; + } + + @Override + protected boolean resolveIndex() { + return false; + } + + @Override + protected SyncedFlushRequest newRequestInstance() { + return new SyncedFlushRequest(); + } + + @Override + protected SyncedFlushRequest newReplicaRequestInstance() { + return new SyncedFlushRequest(); + } + + @Override + protected SyncedFlushResponse newResponseInstance() { + return new SyncedFlushResponse(); + } + + @Override + protected SyncedFlushReplicaResponse newReplicaResponseInstance() { + return new SyncedFlushReplicaResponse(); + } + + @Override + protected String executor() { + return ThreadPool.Names.FLUSH; + } + + @Override + protected ShardIterator shards(ClusterState clusterState, InternalRequest request) { + // get all shards for id + return clusterService.state().routingTable().index(request.concreteIndex()).shard(request.request().shardId().id()).shardsIt(); + } + + @Override + protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { + byte[] commitId = shardRequest.request.commitIds().get(clusterService.localNode().getId()); + IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); + IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); + SyncedFlushResponse syncedFlushResponse = new SyncedFlushResponse(indexShard.syncFlushIfNoPendingChanges(shardRequest.request.syncId(), commitId), shardRequest.request.syncId()); + if (syncedFlushResponse.success() == false) { + throw new ElasticsearchIllegalStateException("could not sync commit on primary"); + } + return new Tuple<>(syncedFlushResponse, shardRequest.request); + } + + @Override + protected SyncedFlushReplicaResponse shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + byte[] commitId = shardRequest.request.commitIds().get(clusterService.localNode().getId()); + IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); + IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); + SyncedFlushReplicaResponse syncedFlushReplicaResponse = new SyncedFlushReplicaResponse(); + boolean success = indexShard.syncFlushIfNoPendingChanges(shardRequest.request.syncId(), commitId); + String message = success ? "synced flush succeeded" : "synced flush failed"; + syncedFlushReplicaResponse.setResult(success, shardRequest.request.index(), shardRequest.shardId.id(), shardRequest.getNodeId(), message); + return syncedFlushReplicaResponse; + } + + + protected SyncedFlushResponse onAllReplicasResponded(SyncedFlushResponse finalResponse, CopyOnWriteArrayList replicaResponses) { + List additionalFailures = new ArrayList<>(); + for (SyncedFlushReplicaResponse replicaResponse : replicaResponses) { + if (replicaResponse.succeeded == false) { + additionalFailures.add(new ActionWriteResponse.ShardInfo.Failure(replicaResponse.getIndex(), replicaResponse.getShardId(), replicaResponse.getNodeId(), replicaResponse.getReason(), RestStatus.CONFLICT, false)); + } + } + additionalFailures.addAll(Arrays.asList(finalResponse.getShardInfo().getFailures())); + finalResponse.setShardInfo(new ActionWriteResponse.ShardInfo(finalResponse.getShardInfo().getTotal(), finalResponse.getShardInfo().getTotal() - additionalFailures.size(), additionalFailures.toArray(new ActionWriteResponse.ShardInfo.Failure[additionalFailures.size()]))); + return finalResponse; + } +} diff --git a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java index 4a945f7e92f..250335d4aaa 100644 --- a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java +++ b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java @@ -19,6 +19,7 @@ package org.elasticsearch.cluster.routing; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -29,18 +30,18 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.test.ElasticsearchAllocationTestCase; import org.junit.Before; import org.junit.Test; -import static org.hamcrest.Matchers.nullValue; - import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; -public class RoutingTableTest extends ElasticsearchAllocationTestCase { +public class RoutingTableTest extends ElasticsearchAllocationTestCase { private static final String TEST_INDEX_1 = "test1"; private static final String TEST_INDEX_2 = "test2"; @@ -72,9 +73,9 @@ public class RoutingTableTest extends ElasticsearchAllocationTestCase { .build(); this.testRoutingTable = new RoutingTable.Builder() - .add(new IndexRoutingTable.Builder(TEST_INDEX_1).initializeAsNew(metaData.index(TEST_INDEX_1)).build()) - .add(new IndexRoutingTable.Builder(TEST_INDEX_2).initializeAsNew(metaData.index(TEST_INDEX_2)).build()) - .build(); + .add(new IndexRoutingTable.Builder(TEST_INDEX_1).initializeAsNew(metaData.index(TEST_INDEX_1)).build()) + .add(new IndexRoutingTable.Builder(TEST_INDEX_2).initializeAsNew(metaData.index(TEST_INDEX_2)).build()) + .build(); this.clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(testRoutingTable).build(); } @@ -82,10 +83,10 @@ public class RoutingTableTest extends ElasticsearchAllocationTestCase { * puts primary shard routings into initializing state */ private void initPrimaries() { - logger.info("adding " + (this.numberOfReplicas + 1) + " nodes and performing rerouting"); + logger.info("adding " + (this.numberOfReplicas + 1) + " nodes and performing rerouting"); Builder discoBuilder = DiscoveryNodes.builder(); - for (int i=0; i commitIds = new HashMap<>(); + final String nodeId = "node_id"; + commitIds.put(nodeId, generateRandomId(randomInt(100))); + SyncedFlushRequest syncedFlushRequest = new SyncedFlushRequest(shardId, randomAsciiOfLength(5), commitIds); + BytesStreamOutput out = new BytesStreamOutput(); + syncedFlushRequest.writeTo(out); + out.close(); + StreamInput in = new BytesStreamInput(out.bytes()); + SyncedFlushRequest request = new SyncedFlushRequest(); + request.readFrom(in); + assertArrayEquals(request.commitIds().get(nodeId), syncedFlushRequest.commitIds().get(nodeId)); + } + + @Test + public void streamSyncResponse() throws InterruptedException, IOException { + ShardRouting shardRouting = new ImmutableShardRouting("test", 0, "test_node", + "other_test_node", randomBoolean(), ShardRoutingState.STARTED, randomInt()); + AtomicReferenceArray atomicReferenceArray = new AtomicReferenceArray(1); + atomicReferenceArray.set(0, new PreSyncedShardFlushResponse(generateRandomId(randomInt(100)), shardRouting)); + PreSyncedFlushResponse preSyncedFlushResponse = new PreSyncedFlushResponse(randomInt(), randomInt(), randomInt(), new ArrayList(), atomicReferenceArray); + BytesStreamOutput out = new BytesStreamOutput(); + preSyncedFlushResponse.writeTo(out); + out.close(); + StreamInput in = new BytesStreamInput(out.bytes()); + PreSyncedFlushResponse request = new PreSyncedFlushResponse(); + request.readFrom(in); + assertArrayEquals(request.commitIds().get(shardRouting), preSyncedFlushResponse.commitIds().get(shardRouting)); + } + + @Test + public void streamShardSyncResponse() throws InterruptedException, IOException { + ShardRouting shardRouting = new ImmutableShardRouting("test", 0, "test_node", + "other_test_node", randomBoolean(), ShardRoutingState.STARTED, randomInt()); + PreSyncedShardFlushResponse preSyncedShardFlushResponse = new PreSyncedShardFlushResponse(generateRandomId(randomInt(100)), shardRouting); + BytesStreamOutput out = new BytesStreamOutput(); + preSyncedShardFlushResponse.writeTo(out); + out.close(); + StreamInput in = new BytesStreamInput(out.bytes()); + PreSyncedShardFlushResponse request = new PreSyncedShardFlushResponse(); + request.readFrom(in); + assertArrayEquals(request.id(), preSyncedShardFlushResponse.id()); + } + + byte[] generateRandomId(int length) { + byte[] id = new byte[length]; + for (int i = 0; i < length; i++) { + id[i] = randomByte(); + } + return id; + } +} From afdab84f2df7c3bec2db81b25432fe76f25f3a98 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 24 Apr 2015 12:38:42 +0200 Subject: [PATCH 04/38] Refactor all actions into inner actions in SyncedFlushService. Add an inflight counter action. Add SyncedFlushResult counter --- .../cluster/routing/RoutingTable.java | 30 - .../elasticsearch/index/engine/Engine.java | 8 +- .../index/engine/InternalEngine.java | 12 +- .../index/engine/ShadowEngine.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 3 +- .../elasticsearch/indices/IndicesModule.java | 1 - .../indices/SyncedFlushService.java | 649 ++++++++++++++++++ .../syncedflush/PreSyncedFlushRequest.java | 70 -- .../syncedflush/PreSyncedFlushResponse.java | 73 -- .../PreSyncedShardFlushRequest.java | 72 -- .../PreSyncedShardFlushResponse.java | 67 -- .../SyncedFlushReplicaResponse.java | 80 --- .../syncedflush/SyncedFlushRequest.java | 89 --- .../syncedflush/SyncedFlushResponse.java | 66 -- .../syncedflush/SyncedFlushService.java | 48 -- .../TransportPreSyncedFlushAction.java | 135 ---- .../TransportSyncedFlushAction.java | 136 ---- .../cluster/routing/RoutingTableTest.java | 18 - .../index/engine/InternalEngineTests.java | 9 +- .../org/elasticsearch/indices/FlushTest.java | 18 +- .../SynceFlushStreamablesTests.java | 93 --- 21 files changed, 679 insertions(+), 1000 deletions(-) create mode 100644 src/main/java/org/elasticsearch/indices/SyncedFlushService.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushRequest.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java delete mode 100644 src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java delete mode 100644 src/test/java/org/elasticsearch/indices/syncedflush/SynceFlushStreamablesTests.java diff --git a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index b6666813ce7..942d512eac8 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -21,7 +21,6 @@ package org.elasticsearch.cluster.routing; import com.carrotsearch.hppc.IntSet; import com.google.common.collect.*; -import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -255,35 +254,6 @@ public class RoutingTable implements Iterable { return new GroupShardsIterator(set); } - /** - * All the shard copies for the provided shard id grouped. Each group is a single element, consisting - * either of the primary shard of one replica. - * - * @param shardId the shard id for the copies we want - * @return All the shard copies (primary and replicas) for the shardId - * @throws IndexMissingException If an index passed does not exists - * @see IndexRoutingTable#groupByAllIt() - */ - public GroupShardsIterator allActiveShardCopiesGrouped(ShardId shardId) throws IndexMissingException { - // use list here since we need to maintain identity across shards - ArrayList set = new ArrayList<>(); - IndexRoutingTable indexRoutingTable = index(shardId.index().name()); - if (indexRoutingTable == null) { - throw new IndexMissingException(new Index(shardId.index().name())); - } - IndexShardRoutingTable copiesRoutingTable = indexRoutingTable.shard(shardId.id()); - if (copiesRoutingTable != null) { - for (ShardRouting shardRouting : copiesRoutingTable) { - if (shardRouting.active()) { - set.add(shardRouting.shardsIt()); - } - } - } else { - throw new ElasticsearchIllegalStateException(shardId + " does not exist"); - } - return new GroupShardsIterator(set); - } - public static Builder builder() { return new Builder(); } diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index c7bfdb11983..cd72d789c70 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -215,7 +215,13 @@ public abstract class Engine implements Closeable { * @param expectedCommitId the expected value of * @return true if the sync commit was made, false o.w. */ - public abstract boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; + public abstract SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; + + public enum SyncedFlushResult { + SUCCESS, + FAILED_COMMIT_MISMATCH, + FAILED_PENDING_OPERATIONS + } final protected GetResult getFromSearcher(Get get) throws EngineException { final Searcher searcher = acquireSearcher("get"); diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 7b05a7449d1..ac2fca4e3da 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -598,26 +598,26 @@ public class InternalEngine extends Engine { } @Override - public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { + public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { // best effort attempt before we acquire locks ensureOpen(); if (indexWriter.hasUncommittedChanges()) { logger.trace("can't sync commit [{}]. have pending changes", syncId); - return false; + return SyncedFlushResult.FAILED_PENDING_OPERATIONS; } if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); - return false; + return SyncedFlushResult.FAILED_COMMIT_MISMATCH; } try (ReleasableLock lock = writeLock.acquire()) { ensureOpen(); if (indexWriter.hasUncommittedChanges()) { logger.trace("can't sync commit [{}]. have pending changes", syncId); - return false; + return SyncedFlushResult.FAILED_PENDING_OPERATIONS; } if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); - return false; + return SyncedFlushResult.FAILED_COMMIT_MISMATCH; } logger.trace("starting sync commit [{}]", syncId); long translogId = translog.currentId(); @@ -628,7 +628,7 @@ public class InternalEngine extends Engine { commitIndexWriter(indexWriter); logger.debug("successfully sync committed. sync id [{}].", syncId); lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); - return true; + return SyncedFlushResult.SUCCESS; } catch (IOException ex) { maybeFailEngine("sync commit", ex); throw new EngineException(shardId, "failed to sync commit", ex); diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index e3a1f60b75f..69887ea042e 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -108,7 +108,7 @@ public class ShadowEngine extends Engine { } @Override - public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index dd1fb3cf619..c04ec70747d 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -116,7 +116,6 @@ import java.io.IOException; import java.io.PrintStream; import java.nio.channels.ClosedByInterruptException; import java.util.Map; -import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -681,7 +680,7 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } - public boolean syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public Engine.SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { verifyStartedOrRecovering(); logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); return engine().syncFlushIfNoPendingChanges(syncId, expectedCommitId); diff --git a/src/main/java/org/elasticsearch/indices/IndicesModule.java b/src/main/java/org/elasticsearch/indices/IndicesModule.java index a6080d5a4a1..785d2af7eca 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -38,7 +38,6 @@ import org.elasticsearch.indices.recovery.RecoverySource; import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; -import org.elasticsearch.indices.syncedflush.SyncedFlushService; import org.elasticsearch.indices.ttl.IndicesTTLService; /** diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java new file mode 100644 index 00000000000..a0648c31b9a --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -0,0 +1,649 @@ +/* + * 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; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexShardMissingException; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardException; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +public class SyncedFlushService extends AbstractComponent { + + // nocommmit: check these are ok + public static final String PRE_SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/pre"; + public static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; + public static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; + + public static final String SETTING_PRE_SYNC_TIMEOUT = "indices.flush.synced.presync_timeout"; + public static final String SETTING_SYNC_TIMEOUT = "indices.flush.synced.sync_timeout"; + public static final String SETTING_IN_FLIGHT_OPS_TIMEOUT = "indices.flush.synced.in_flight_ops_timeout"; + + private final IndicesService indicesService; + private final ClusterService clusterService; + private final TransportService transportService; + private final TimeValue preSyncTimeout; + private final TimeValue syncTimeout; + private final TimeValue inflightOpsTimeout; + + @Inject + public SyncedFlushService(Settings settings, IndicesService indicesService, ClusterService clusterService, TransportService transportService) { + super(settings); + this.indicesService = indicesService; + this.clusterService = clusterService; + this.transportService = transportService; + + transportService.registerHandler(PRE_SYNCED_FLUSH_ACTION_NAME, new PreSyncedFlushTransportHandler()); + transportService.registerHandler(SYNCED_FLUSH_ACTION_NAME, new SyncedFlushTransportHandler()); + transportService.registerHandler(IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpCountTransportHandler()); + preSyncTimeout = settings.getAsTime(SETTING_PRE_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); + syncTimeout = settings.getAsTime(SETTING_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); + inflightOpsTimeout = settings.getAsTime(SETTING_IN_FLIGHT_OPS_TIMEOUT, TimeValue.timeValueMinutes(5)); + } + + public SyncedFlushResult attemptSyncedFlush(ShardId shardId) throws ExecutionException, InterruptedException { + final ClusterState state = clusterService.state(); + final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); + if (indexRoutingTable == null) { + throw new IndexMissingException(shardId.index()); + } + final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); + if (shardRoutingTable == null) { + throw new IndexShardMissingException(shardId); + } + final List activeShards = shardRoutingTable.activeShards(); + Map commitIds = sendPreSyncRequests(activeShards, state, shardId); + + if (commitIds.isEmpty()) { + return new SyncedFlushResult("all shards failed to commit on pre-sync"); + } + + int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); + if (inflight < 0 || inflight > 0) { + return new SyncedFlushResult("operation counter on primary is non zero [" + inflight + "]"); + } + + String syncId = Strings.base64UUID(); + Map results = sendSyncRequests(syncId, activeShards, state, commitIds, shardId); + + return new SyncedFlushResult(syncId, results); + } + + /** returns the number of inflight operations on primary. -1 upon error. */ + protected int getInflightOpsCount(final ShardId shardId, ClusterState state, IndexShardRoutingTable shardRoutingTable) { + final ShardRouting primaryShard = shardRoutingTable.primaryShard(); + final DiscoveryNode primaryNode = state.nodes().get(primaryShard.currentNodeId()); + if (primaryNode == null) { + logger.trace("{} failed to resolve node for primary shard {}, skipping sync", shardId, primaryShard); + return -1; + } + final AtomicInteger result = new AtomicInteger(-1); + final CountDownLatch latch = new CountDownLatch(1); + transportService.sendRequest(primaryNode, IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpsRequest(shardId), + new BaseTransportResponseHandler() { + @Override + public InFlightOpsResponse newInstance() { + return new InFlightOpsResponse(); + } + + @Override + public void handleResponse(InFlightOpsResponse response) { + result.set(response.opCount()); + latch.countDown(); + } + + @Override + public void handleException(TransportException exp) { + logger.debug("{} unexpected error while retrieving inflight op count", shardId); + result.set(-1); + latch.countDown(); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + try { + if (latch.await(inflightOpsTimeout.millis(), TimeUnit.MILLISECONDS) == false) { + logger.debug("{} in flight operation check timed out after [{}]", shardId, syncTimeout); + } + } catch (InterruptedException e) { + logger.debug("{} interrupted while waiting for in flight operation check", shardId); + } + + final int count = result.get(); + logger.trace("{} in flight operation count [{}]", shardId, count); + return count; + } + + + private Map sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId) { + final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); + final Map results = ConcurrentCollections.newConcurrentMap(); + for (final ShardRouting shard : shards) { + final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); + if (node == null) { + logger.trace("{} is assigned to an unknown node. skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); + results.put(shard, new SyncedFlushResponse("unknown node")); + countDownLatch.countDown(); + continue; + } + final byte[] expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); + if (expectedCommitId == null) { + logger.trace("{} can't resolve expected commit id for {}, skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); + results.put(shard, new SyncedFlushResponse("no commit id from pre-sync flush")); + countDownLatch.countDown(); + continue; + } + logger.trace("{} sending synced flush request to {}. sync id [{}].", shardId, shard, syncId); + transportService.sendRequest(node, SYNCED_FLUSH_ACTION_NAME, new SyncedFlushRequest(shard.shardId(), syncId, expectedCommitId), + new BaseTransportResponseHandler() { + @Override + public SyncedFlushResponse newInstance() { + return new SyncedFlushResponse(); + } + + @Override + public void handleResponse(SyncedFlushResponse response) { + SyncedFlushResponse existing = results.put(shard, response); + assert existing == null : "got two answers for node [" + node + "]"; + // count after the assert so we won't decrement twice in handleException + countDownLatch.countDown(); + } + + @Override + public void handleException(TransportException exp) { + logger.trace("{} error while performing synced flush on [{}], skipping", exp, shardId, shard); + results.put(shard, new SyncedFlushResponse(exp.getMessage())); + countDownLatch.countDown(); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + } + try { + if (countDownLatch.await(syncTimeout.millis(), TimeUnit.MILLISECONDS) == false) { + logger.debug("{} waiting for synced flush with id [{}] timed out after [{}]. pending ops [{}]", shardId, syncId, syncTimeout, countDownLatch.getCount()); + } + } catch (InterruptedException e) { + logger.debug("{} interrupted while waiting for sync requests (sync id [{}])", shardId, syncId); + } + + return results; + } + + /** send presync requests to all started copies of the given shard */ + Map sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId) { + final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); + final Map commitIds = ConcurrentCollections.newConcurrentMap(); + for (final ShardRouting shard : shards) { + logger.trace("{} sending pre-synced flush request to {}", shardId, shard); + final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); + if (node == null) { + logger.trace("{} shard routing {} refers to an unknown node. skipping.", shardId, shard); + countDownLatch.countDown(); + continue; + } + transportService.sendRequest(node, PRE_SYNCED_FLUSH_ACTION_NAME, new PreSyncedFlushRequest(shard.shardId()), new BaseTransportResponseHandler() { + @Override + public PreSyncedFlushResponse newInstance() { + return new PreSyncedFlushResponse(); + } + + @Override + public void handleResponse(PreSyncedFlushResponse response) { + byte[] existing = commitIds.put(node.id(), response.commitId()); + assert existing == null : "got two answers for node [" + node + "]"; + // count after the assert so we won't decrement twice in handleException + countDownLatch.countDown(); + } + + @Override + public void handleException(TransportException exp) { + logger.trace("{} error while performing pre synced flush on [{}], skipping", shardId, exp, shard); + countDownLatch.countDown(); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + } + try { + if (countDownLatch.await(preSyncTimeout.millis(), TimeUnit.MILLISECONDS) == false) { + logger.debug("{} waiting for pre sync flush requests timed out after [{}]. pending ops [{}]", shardId, preSyncTimeout, countDownLatch.getCount()); + } + } catch (InterruptedException e) { + logger.debug("{} interrupted while waiting for presync requests", shardId); + } + + return commitIds; + } + + private PreSyncedFlushResponse performPreSyncedFlush(PreSyncedFlushRequest request) { + IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); + FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); + logger.trace("{} performing pre sync flush", request.shardId()); + byte[] id = indexShard.flush(flushRequest); + logger.trace("{} pre sync flush done. commit id {}", request.shardId(), id); + return new PreSyncedFlushResponse(id); + } + + private SyncedFlushResponse performSyncedFlush(SyncedFlushRequest request) { + IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); + IndexShard indexShard = indexService.shardSafe(request.shardId().id()); + logger.trace("{} performing sync flush. sync id [{}], expected commit id {}", request.shardId(), request.syncId(), request.expectedCommitId()); + Engine.SyncedFlushResult result = indexShard.syncFlushIfNoPendingChanges(request.syncId(), request.expectedCommitId()); + logger.trace("{} sync flush done. sync id [{}], result [{}]", request.shardId(), request.syncId(), result); + switch (result) { + case SUCCESS: + return new SyncedFlushResponse(); + case FAILED_COMMIT_MISMATCH: + return new SyncedFlushResponse("commit has changed"); + case FAILED_PENDING_OPERATIONS: + return new SyncedFlushResponse("pending operations"); + default: + throw new ElasticsearchException("unknown synced flush result [" + result + "]"); + } + } + + private InFlightOpsResponse performInFlightOps(InFlightOpsRequest request) { + IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); + IndexShard indexShard = indexService.shardSafe(request.shardId().id()); + if (indexShard.routingEntry().primary() == false) { + throw new IndexShardException(request.shardId(), "expected a primary shard"); + } + // nocommit - implement :) + int opCount = 0; + logger.trace("{} in flight operations sampled at [{}]", request.shardId(), opCount); + return new InFlightOpsResponse(opCount); + } + + public static class SyncedFlushResult { + private final String failureReason; + private final Map shardResponses; + private final String syncId; + + /** failure constructor */ + SyncedFlushResult(String failureReason) { + this.syncId = null; + this.failureReason = failureReason; + this.shardResponses = new HashMap<>(); + } + + /** success constructor */ + SyncedFlushResult(String syncId, Map shardResponses) { + this.failureReason = null; + this.shardResponses = shardResponses; + this.syncId = syncId; + } + + public boolean success() { + return syncId != null; + } + + public String failureReason() { + return failureReason; + } + + public String syncId() { + return syncId; + } + + /** total number of shards for which a sync attempt was made */ + public int totalShards() { + return shardResponses.size(); + } + + public int successfulShards() { + int i = 0; + for (SyncedFlushResponse result : shardResponses.values()) { + if (result.success()) { + i++; + } + } + return i; + } + + public Map shardResponses() { + return shardResponses; + } + + } + + final static class PreSyncedFlushRequest extends TransportRequest { + private ShardId shardId; + + + PreSyncedFlushRequest() { + } + + public PreSyncedFlushRequest(ShardId shardId) { + this.shardId = shardId; + } + + @Override + public String toString() { + return "PreSyncedFlushRequest{" + + "shardId=" + shardId + + '}'; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + this.shardId = ShardId.readShardId(in); + } + + public ShardId shardId() { + return shardId; + } + } + + final static class PreSyncedFlushResponse extends TransportResponse { + + private byte[] commitId; + + PreSyncedFlushResponse() { + } + + PreSyncedFlushResponse(byte[] commitId) { + this.commitId = commitId; + } + + public byte[] commitId() { + return commitId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + commitId = in.readByteArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeByteArray(commitId); + } + } + + static final class SyncedFlushRequest extends TransportRequest { + + private String syncId; + private byte[] expectedCommitId; + private ShardId shardId; + + public SyncedFlushRequest() { + } + + public SyncedFlushRequest(ShardId shardId, String syncId, byte[] expectedCommitId) { + this.expectedCommitId = expectedCommitId; + this.shardId = shardId; + this.syncId = syncId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + expectedCommitId = in.readByteArray(); + syncId = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + out.writeByteArray(expectedCommitId); + out.writeString(syncId); + } + + public ShardId shardId() { + return shardId; + } + + public String syncId() { + return syncId; + } + + public byte[] expectedCommitId() { + return expectedCommitId; + } + + @Override + public String toString() { + return "SyncedFlushRequest{" + + "shardId=" + shardId + + ",syncId='" + syncId + '\'' + + '}'; + } + } + + static final class SyncedFlushResponse extends TransportResponse { + + /** a non null value indicates a failure to sync flush. null means success */ + String failureReason; + + + public SyncedFlushResponse() { + failureReason = null; + } + + public SyncedFlushResponse(String failureReason) { + this.failureReason = failureReason; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + failureReason = in.readOptionalString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalString(failureReason); + } + + public boolean success() { + return failureReason == null; + } + + public String failureReason() { + return failureReason; + } + + @Override + public String toString() { + return "SyncedFlushResponse{" + + "success=" + success() + + ", failureReason='" + failureReason + '\'' + + '}'; + } + } + + + static final class InFlightOpsRequest extends TransportRequest { + + private ShardId shardId; + + public InFlightOpsRequest() { + } + + public InFlightOpsRequest(ShardId shardId) { + this.shardId = shardId; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + public ShardId shardId() { + return shardId; + } + + @Override + public String toString() { + return "InFlightOpsRequest{" + + "shardId=" + shardId + + '}'; + } + } + + static final class InFlightOpsResponse extends TransportResponse { + + int opCount; + + + public InFlightOpsResponse() { + } + + public InFlightOpsResponse(int opCount) { + this.opCount = opCount; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + opCount = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(opCount); + } + + public int opCount() { + return opCount; + } + + @Override + public String toString() { + return "InFlightOpsResponse{" + + "opCount=" + opCount + + '}'; + } + } + + + private class PreSyncedFlushTransportHandler extends BaseTransportRequestHandler { + + @Override + public PreSyncedFlushRequest newInstance() { + return new PreSyncedFlushRequest(); + } + + @Override + public void messageReceived(PreSyncedFlushRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performPreSyncedFlush(request)); + } + + @Override + public String executor() { + return ThreadPool.Names.FLUSH; + } + } + + + private class SyncedFlushTransportHandler extends BaseTransportRequestHandler { + + @Override + public SyncedFlushRequest newInstance() { + return new SyncedFlushRequest(); + } + + @Override + public void messageReceived(SyncedFlushRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performSyncedFlush(request)); + } + + @Override + public String executor() { + return ThreadPool.Names.FLUSH; + } + } + + private class InFlightOpCountTransportHandler extends BaseTransportRequestHandler { + + @Override + public InFlightOpsRequest newInstance() { + return new InFlightOpsRequest(); + } + + @Override + public void messageReceived(InFlightOpsRequest request, TransportChannel channel) throws Exception { + channel.sendResponse(performInFlightOps(request)); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + +} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushRequest.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushRequest.java deleted file mode 100644 index 97401bf1cfb..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushRequest.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.shard.ShardId; - -import java.io.IOException; -import java.util.Arrays; - -/** - */ -public class PreSyncedFlushRequest extends BroadcastOperationRequest { - private ShardId shardId; - - - PreSyncedFlushRequest() { - } - - public PreSyncedFlushRequest(ShardId shardId) { - super(Arrays.asList(shardId.getIndex()).toArray(new String[0])); - this.shardId = shardId; - } - - @Override - public String toString() { - return "PreSyncedFlushRequest{" + - "shardId=" + shardId + - '}'; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - shardId.writeTo(out); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - this.shardId = ShardId.readShardId(in); - } - - public ShardId shardId() { - return shardId; - } - - public void shardId(ShardId shardId) { - this.shardId = shardId; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java deleted file mode 100644 index c757f2f669a..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedFlushResponse.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.ShardOperationFailedException; -import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReferenceArray; - -/** - * A response to pre synced flush action. - */ -public class PreSyncedFlushResponse extends BroadcastOperationResponse { - - Map commitIds = new HashMap<>(); - - PreSyncedFlushResponse() { - } - - public PreSyncedFlushResponse(int totalShards, int successfulShards, int failedShards, List shardFailures, AtomicReferenceArray shardsResponses) { - super(totalShards, successfulShards, failedShards, shardFailures); - for (int i = 0; i < shardsResponses.length(); i++) { - PreSyncedShardFlushResponse preSyncedShardFlushResponse = (PreSyncedShardFlushResponse) shardsResponses.get(i); - commitIds.put(preSyncedShardFlushResponse.shardRouting().currentNodeId(), preSyncedShardFlushResponse.id()); - } - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - int numCommitIds = in.readVInt(); - for (int i = 0; i < numCommitIds; i++) { - commitIds.put(in.readString(), in.readByteArray()); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeVInt(commitIds.size()); - for (Map.Entry entry : commitIds.entrySet()) { - out.writeString(entry.getKey()); - out.writeByteArray(entry.getValue()); - } - } - - public Map commitIds() { - return commitIds; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java deleted file mode 100644 index 4fa3cbd3c33..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushRequest.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest; -import org.elasticsearch.cluster.routing.ImmutableShardRouting; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - * - */ -class PreSyncedShardFlushRequest extends BroadcastShardOperationRequest { - - private ShardRouting shardRouting; - // we need our own request because it has to include the shard routing - private PreSyncedFlushRequest request = new PreSyncedFlushRequest(); - - PreSyncedShardFlushRequest() { - } - - PreSyncedShardFlushRequest(ShardRouting shardRouting, PreSyncedFlushRequest request) { - super(shardRouting.shardId(), request); - this.request = request; - this.shardRouting = shardRouting; - } - - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - request.readFrom(in); - shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); - - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - request.writeTo(out); - shardRouting.writeTo(out); - - } - - PreSyncedFlushRequest getRequest() { - return request; - } - - public ShardRouting shardRouting() { - return shardRouting; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java deleted file mode 100644 index 55f166f3a18..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/PreSyncedShardFlushResponse.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse; -import org.elasticsearch.cluster.routing.ImmutableShardRouting; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - * - */ -class PreSyncedShardFlushResponse extends BroadcastShardOperationResponse { - byte[] id; - private ShardRouting shardRouting; - - PreSyncedShardFlushResponse() { - } - - PreSyncedShardFlushResponse(byte[] id, ShardRouting shardRouting) { - super(shardRouting.shardId()); - this.id = id; - this.shardRouting = shardRouting; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - id = in.readByteArray(); - shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeByteArray(id); - shardRouting.writeTo(out); - } - - byte[] id() { - return id; - } - - public ShardRouting shardRouting() { - return shardRouting; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java deleted file mode 100644 index a08edb635dc..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushReplicaResponse.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.transport.TransportResponse; - -import java.io.IOException; - -public class SyncedFlushReplicaResponse extends TransportResponse { - boolean succeeded = true; - private String index; - private int shardId; - private String nodeId; - private String reason; - - void setResult(boolean succeeded, String index, int shardId, String nodeId, String reason) { - this.succeeded = succeeded; - this.index = index; - this.shardId = shardId; - this.nodeId = nodeId; - this.reason = reason; - } - - public String getIndex() { - return index; - } - - public int getShardId() { - return shardId; - } - - public String getNodeId() { - return nodeId; - } - - public String getReason() { - return reason; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - succeeded = in.readBoolean(); - this.index = in.readString(); - this.shardId = in.readInt(); - this.nodeId = in.readString(); - this.reason = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(succeeded); - out.writeString(index); - out.writeInt(shardId); - out.writeString(nodeId); - out.writeString(reason); - } - -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java deleted file mode 100644 index 10ba56d0f65..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushRequest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.support.replication.ShardReplicationOperationRequest; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.shard.ShardId; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -/** - */ -public class SyncedFlushRequest extends ShardReplicationOperationRequest { - - private String syncId; - private Map commitIds; - private ShardId shardId; - - public SyncedFlushRequest() { - } - - public SyncedFlushRequest(ShardId shardId, String syncId, Map commitIds) { - this.commitIds = commitIds; - this.shardId = shardId; - this.syncId = syncId; - this.index(shardId.index().getName()); - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - shardId = ShardId.readShardId(in); - commitIds = new HashMap<>(); - int numCommitIds = in.readVInt(); - for (int i = 0; i < numCommitIds; i++) { - commitIds.put(in.readString(), in.readByteArray()); - } - syncId = in.readString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - shardId.writeTo(out); - out.writeVInt(commitIds.size()); - for (Map.Entry entry : commitIds.entrySet()) { - out.writeString(entry.getKey()); - out.writeByteArray(entry.getValue()); - } - out.writeString(syncId); - } - - @Override - public String toString() { - return "write sync commit {" + shardId + "}"; - } - - public ShardId shardId() { - return shardId; - } - - public String syncId() { - return syncId; - } - - public Map commitIds() { - return commitIds; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java deleted file mode 100644 index 199c7d14978..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushResponse.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.ActionWriteResponse; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; - -import java.io.IOException; - -/** - */ -public class SyncedFlushResponse extends ActionWriteResponse { - - - private boolean succes; - - String syncCommitId; - - public SyncedFlushResponse() { - - } - - public SyncedFlushResponse(boolean success, String syncCommitId) { - this.succes = success; - this.syncCommitId = syncCommitId; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - this.succes = in.readBoolean(); - syncCommitId = in.readOptionalString(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeBoolean(succes); - out.writeOptionalString(syncCommitId); - } - - public boolean success() { - return succes; - } - public String getSyncId() { - return syncCommitId; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java deleted file mode 100644 index 5920ef7cf95..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/SyncedFlushService.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.shard.ShardId; - -import java.util.concurrent.ExecutionException; - -public class SyncedFlushService extends AbstractComponent { - - private final TransportPreSyncedFlushAction transportPreSyncedFlushAction; - private final TransportSyncedFlushAction transportSyncedFlushAction; - - @Inject - public SyncedFlushService(Settings settings, TransportPreSyncedFlushAction transportPreSyncedFlushAction, TransportSyncedFlushAction transportSyncedFlushAction) { - super(settings); - this.transportPreSyncedFlushAction = transportPreSyncedFlushAction; - this.transportSyncedFlushAction = transportSyncedFlushAction; - } - - public SyncedFlushResponse attemptSyncedFlush(ShardId shardId) throws ExecutionException, InterruptedException { - PreSyncedFlushResponse preSyncedFlushResponse = transportPreSyncedFlushAction.execute(new PreSyncedFlushRequest(shardId)).get(); - // exit if this did not work - String syncId = Strings.base64UUID(); - SyncedFlushResponse syncedFlushResponse = transportSyncedFlushAction.execute(new SyncedFlushRequest(shardId, syncId, preSyncedFlushResponse.commitIds())).get(); - return syncedFlushResponse; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java b/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java deleted file mode 100644 index 7a76136fb95..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/TransportPreSyncedFlushAction.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ShardOperationFailedException; -import org.elasticsearch.action.admin.indices.flush.FlushRequest; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.DefaultShardOperationFailedException; -import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException; -import org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -import java.util.List; -import java.util.concurrent.atomic.AtomicReferenceArray; - -import static com.google.common.collect.Lists.newArrayList; - - -/** - * Sync Commit Action. - */ -public class TransportPreSyncedFlushAction extends TransportBroadcastOperationAction { - - private final IndicesService indicesService; - - public static final String NAME = "indices:admin/presyncedflush"; - - @Inject - public TransportPreSyncedFlushAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService, ActionFilters actionFilters) { - super(settings, NAME, threadPool, clusterService, transportService, actionFilters); - this.indicesService = indicesService; - } - - @Override - protected String executor() { - return ThreadPool.Names.FLUSH; - } - - @Override - protected PreSyncedFlushRequest newRequestInstance() { - return new PreSyncedFlushRequest(); - } - - @Override - protected PreSyncedFlushResponse newResponse(PreSyncedFlushRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { - int successfulShards = 0; - int failedShards = 0; - List shardFailures = null; - for (int i = 0; i < shardsResponses.length(); i++) { - Object shardResponse = shardsResponses.get(i); - if (shardResponse == null) { - // a non active shard, ignore - } else if (shardResponse instanceof BroadcastShardOperationFailedException) { - failedShards++; - if (shardFailures == null) { - shardFailures = newArrayList(); - } - shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse)); - } else { - successfulShards++; - } - } - return new PreSyncedFlushResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, shardsResponses); - } - - @Override - protected PreSyncedShardFlushRequest newShardRequest() { - return new PreSyncedShardFlushRequest(); - } - - @Override - protected PreSyncedShardFlushRequest newShardRequest(int numShards, ShardRouting shard, PreSyncedFlushRequest request) { - return new PreSyncedShardFlushRequest(shard, request); - } - - @Override - protected PreSyncedShardFlushResponse newShardResponse() { - return new PreSyncedShardFlushResponse(); - } - - @Override - protected PreSyncedShardFlushResponse shardOperation(PreSyncedShardFlushRequest request) throws ElasticsearchException { - IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); - FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); - byte[] id = indexShard.flush(flushRequest); - return new PreSyncedShardFlushResponse(id, request.shardRouting()); - } - - /** - * The sync commit request works against one primary and all of its copies. - */ - @Override - protected GroupShardsIterator shards(ClusterState clusterState, PreSyncedFlushRequest request, String[] concreteIndices) { - return clusterState.routingTable().allActiveShardCopiesGrouped(request.shardId()); - } - - @Override - protected ClusterBlockException checkGlobalBlock(ClusterState state, PreSyncedFlushRequest request) { - return null; - } - - @Override - protected ClusterBlockException checkRequestBlock(ClusterState state, PreSyncedFlushRequest countRequest, String[] concreteIndices) { - return null; - } -} diff --git a/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java b/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java deleted file mode 100644 index b8213a13226..00000000000 --- a/src/main/java/org/elasticsearch/indices/syncedflush/TransportSyncedFlushAction.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.ElasticsearchIllegalStateException; -import org.elasticsearch.action.ActionWriteResponse; -import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.replication.TransportShardReplicationOperationAction; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.action.shard.ShardStateAction; -import org.elasticsearch.cluster.routing.ShardIterator; -import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportService; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; - -/** - */ -public class TransportSyncedFlushAction extends TransportShardReplicationOperationAction { - - public static final String NAME = "indices:admin/syncedflush"; - - @Inject - public TransportSyncedFlushAction(Settings settings, TransportService transportService, ClusterService clusterService, - IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, - ActionFilters actionFilters) { - super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters); - } - - @Override - protected boolean checkWriteConsistency() { - return true; - } - - @Override - protected boolean resolveIndex() { - return false; - } - - @Override - protected SyncedFlushRequest newRequestInstance() { - return new SyncedFlushRequest(); - } - - @Override - protected SyncedFlushRequest newReplicaRequestInstance() { - return new SyncedFlushRequest(); - } - - @Override - protected SyncedFlushResponse newResponseInstance() { - return new SyncedFlushResponse(); - } - - @Override - protected SyncedFlushReplicaResponse newReplicaResponseInstance() { - return new SyncedFlushReplicaResponse(); - } - - @Override - protected String executor() { - return ThreadPool.Names.FLUSH; - } - - @Override - protected ShardIterator shards(ClusterState clusterState, InternalRequest request) { - // get all shards for id - return clusterService.state().routingTable().index(request.concreteIndex()).shard(request.request().shardId().id()).shardsIt(); - } - - @Override - protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { - byte[] commitId = shardRequest.request.commitIds().get(clusterService.localNode().getId()); - IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); - IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); - SyncedFlushResponse syncedFlushResponse = new SyncedFlushResponse(indexShard.syncFlushIfNoPendingChanges(shardRequest.request.syncId(), commitId), shardRequest.request.syncId()); - if (syncedFlushResponse.success() == false) { - throw new ElasticsearchIllegalStateException("could not sync commit on primary"); - } - return new Tuple<>(syncedFlushResponse, shardRequest.request); - } - - @Override - protected SyncedFlushReplicaResponse shardOperationOnReplica(ReplicaOperationRequest shardRequest) { - byte[] commitId = shardRequest.request.commitIds().get(clusterService.localNode().getId()); - IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); - IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); - SyncedFlushReplicaResponse syncedFlushReplicaResponse = new SyncedFlushReplicaResponse(); - boolean success = indexShard.syncFlushIfNoPendingChanges(shardRequest.request.syncId(), commitId); - String message = success ? "synced flush succeeded" : "synced flush failed"; - syncedFlushReplicaResponse.setResult(success, shardRequest.request.index(), shardRequest.shardId.id(), shardRequest.getNodeId(), message); - return syncedFlushReplicaResponse; - } - - - protected SyncedFlushResponse onAllReplicasResponded(SyncedFlushResponse finalResponse, CopyOnWriteArrayList replicaResponses) { - List additionalFailures = new ArrayList<>(); - for (SyncedFlushReplicaResponse replicaResponse : replicaResponses) { - if (replicaResponse.succeeded == false) { - additionalFailures.add(new ActionWriteResponse.ShardInfo.Failure(replicaResponse.getIndex(), replicaResponse.getShardId(), replicaResponse.getNodeId(), replicaResponse.getReason(), RestStatus.CONFLICT, false)); - } - } - additionalFailures.addAll(Arrays.asList(finalResponse.getShardInfo().getFailures())); - finalResponse.setShardInfo(new ActionWriteResponse.ShardInfo(finalResponse.getShardInfo().getTotal(), finalResponse.getShardInfo().getTotal() - additionalFailures.size(), additionalFailures.toArray(new ActionWriteResponse.ShardInfo.Failure[additionalFailures.size()]))); - return finalResponse; - } -} diff --git a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java index 250335d4aaa..f6f6c6cc38e 100644 --- a/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java +++ b/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTest.java @@ -19,7 +19,6 @@ package org.elasticsearch.cluster.routing; -import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -30,7 +29,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.test.ElasticsearchAllocationTestCase; import org.junit.Before; @@ -246,20 +244,4 @@ public class RoutingTableTest extends ElasticsearchAllocationTestCase { fail("Calling with non-existing index should be ignored at the moment"); } } - - @Test - public void testAllActiveShardCopiesGrouped() { - assertThat(this.testRoutingTable.allActiveShardCopiesGrouped(new ShardId(TEST_INDEX_1, randomInt(numberOfShards - 1))).size(), is(0)); - try { - this.testRoutingTable.allActiveShardCopiesGrouped(new ShardId(TEST_INDEX_1, numberOfShards)).size(); - fail(); - } catch (ElasticsearchIllegalStateException e) { - } - initPrimaries(); - assertThat(this.testRoutingTable.allActiveShardCopiesGrouped(new ShardId(TEST_INDEX_1, randomInt(numberOfShards - 1))).size(), is(0)); - startInitializingShards(TEST_INDEX_1); - assertThat(this.testRoutingTable.allActiveShardCopiesGrouped(new ShardId(TEST_INDEX_1, randomInt(numberOfShards - 1))).size(), is(1)); - startInitializingShards(TEST_INDEX_1); - assertThat(this.testRoutingTable.allActiveShardCopiesGrouped(new ShardId(TEST_INDEX_1, randomInt(numberOfShards - 1))).size(), is(numberOfReplicas + 1)); - } } diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index c05467ea5a3..ccc84c2caf7 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -695,11 +695,14 @@ public class InternalEngineTests extends ElasticsearchTestCase { assertThat(commitID, equalTo(store.readLastCommittedSegmentsInfo().getId())); byte[] fakeId = commitID.clone(); fakeId[0] = (byte) ~fakeId[0]; - assertFalse("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", fakeId)); + assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", fakeId), + equalTo(Engine.SyncedFlushResult.FAILED_COMMIT_MISMATCH)); engine.create(new Engine.Create(null, newUid("2"), doc)); - assertFalse("should fail to sync flush with right id but pending doc", engine.syncFlushIfNoPendingChanges(syncId + "2", commitID)); + assertThat("should fail to sync flush with right id but pending doc", engine.syncFlushIfNoPendingChanges(syncId + "2", commitID), + equalTo(Engine.SyncedFlushResult.FAILED_PENDING_OPERATIONS)); commitID = engine.flush(); - assertTrue("should succeed to flush commit with right id and no pending doc", engine.syncFlushIfNoPendingChanges(syncId, commitID)); + assertThat("should succeed to flush commit with right id and no pending doc", engine.syncFlushIfNoPendingChanges(syncId, commitID), + equalTo(Engine.SyncedFlushResult.SUCCESS)); assertThat(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); assertThat(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); } diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index f04e30f2d48..a4a9b4d2171 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -19,16 +19,14 @@ package org.elasticsearch.indices; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.syncedflush.SyncedFlushResponse; -import org.elasticsearch.indices.syncedflush.SyncedFlushService; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; import java.io.IOException; @@ -79,6 +77,7 @@ public class FlushTest extends ElasticsearchIntegrationTest { } } + @TestLogging("indices:TRACE") public void testSyncedFlush() throws ExecutionException, InterruptedException, IOException { internalCluster().ensureAtLeastNumDataNodes(2); prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).get(); @@ -89,15 +88,16 @@ public class FlushTest extends ElasticsearchIntegrationTest { assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); } - ClusterStateResponse state = client().admin().cluster().prepareState().get(); - String nodeId = state.getState().getRoutingTable().index("test").shard(0).getShards().get(0).currentNodeId(); - SyncedFlushResponse syncedFlushResponse = internalCluster().getInstance(SyncedFlushService.class).attemptSyncedFlush(new ShardId("test", 0)); - assertTrue(syncedFlushResponse.success()); + SyncedFlushService.SyncedFlushResult result = internalCluster().getInstance(SyncedFlushService.class).attemptSyncedFlush(new ShardId("test", 0)); + assertTrue(result.success()); + assertThat(result.totalShards(), equalTo(indexStats.getShards().length)); + assertThat(result.successfulShards(), equalTo(indexStats.getShards().length)); indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); - assertThat(indexStats.getShards().length, equalTo(client().admin().indices().prepareGetIndex().get().getSettings().get("test").getAsInt(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, -1) + 1)); + String syncId = result.syncId(); for (ShardStats shardStats : indexStats.getShards()) { - assertThat(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncedFlushResponse.getSyncId())); + final String shardSyncId = shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID); + assertThat(shardSyncId, equalTo(syncId)); } } diff --git a/src/test/java/org/elasticsearch/indices/syncedflush/SynceFlushStreamablesTests.java b/src/test/java/org/elasticsearch/indices/syncedflush/SynceFlushStreamablesTests.java deleted file mode 100644 index 29c21e76d92..00000000000 --- a/src/test/java/org/elasticsearch/indices/syncedflush/SynceFlushStreamablesTests.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.syncedflush; - -import org.elasticsearch.action.ShardOperationFailedException; -import org.elasticsearch.cluster.routing.ImmutableShardRouting; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.common.io.stream.BytesStreamInput; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.test.ElasticsearchTestCase; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReferenceArray; - -public class SynceFlushStreamablesTests extends ElasticsearchTestCase { - - @Test - public void streamWriteSyncResponse() throws InterruptedException, IOException { - ShardId shardId = new ShardId("test", 0); - Map commitIds = new HashMap<>(); - final String nodeId = "node_id"; - commitIds.put(nodeId, generateRandomId(randomInt(100))); - SyncedFlushRequest syncedFlushRequest = new SyncedFlushRequest(shardId, randomAsciiOfLength(5), commitIds); - BytesStreamOutput out = new BytesStreamOutput(); - syncedFlushRequest.writeTo(out); - out.close(); - StreamInput in = new BytesStreamInput(out.bytes()); - SyncedFlushRequest request = new SyncedFlushRequest(); - request.readFrom(in); - assertArrayEquals(request.commitIds().get(nodeId), syncedFlushRequest.commitIds().get(nodeId)); - } - - @Test - public void streamSyncResponse() throws InterruptedException, IOException { - ShardRouting shardRouting = new ImmutableShardRouting("test", 0, "test_node", - "other_test_node", randomBoolean(), ShardRoutingState.STARTED, randomInt()); - AtomicReferenceArray atomicReferenceArray = new AtomicReferenceArray(1); - atomicReferenceArray.set(0, new PreSyncedShardFlushResponse(generateRandomId(randomInt(100)), shardRouting)); - PreSyncedFlushResponse preSyncedFlushResponse = new PreSyncedFlushResponse(randomInt(), randomInt(), randomInt(), new ArrayList(), atomicReferenceArray); - BytesStreamOutput out = new BytesStreamOutput(); - preSyncedFlushResponse.writeTo(out); - out.close(); - StreamInput in = new BytesStreamInput(out.bytes()); - PreSyncedFlushResponse request = new PreSyncedFlushResponse(); - request.readFrom(in); - assertArrayEquals(request.commitIds().get(shardRouting), preSyncedFlushResponse.commitIds().get(shardRouting)); - } - - @Test - public void streamShardSyncResponse() throws InterruptedException, IOException { - ShardRouting shardRouting = new ImmutableShardRouting("test", 0, "test_node", - "other_test_node", randomBoolean(), ShardRoutingState.STARTED, randomInt()); - PreSyncedShardFlushResponse preSyncedShardFlushResponse = new PreSyncedShardFlushResponse(generateRandomId(randomInt(100)), shardRouting); - BytesStreamOutput out = new BytesStreamOutput(); - preSyncedShardFlushResponse.writeTo(out); - out.close(); - StreamInput in = new BytesStreamInput(out.bytes()); - PreSyncedShardFlushResponse request = new PreSyncedShardFlushResponse(); - request.readFrom(in); - assertArrayEquals(request.id(), preSyncedShardFlushResponse.id()); - } - - byte[] generateRandomId(int length) { - byte[] id = new byte[length]; - for (int i = 0; i < length; i++) { - id[i] = randomByte(); - } - return id; - } -} From 8b7b2f3cdf632ca41f9164e03b00aa6baea1090e Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Mon, 4 May 2015 10:52:20 +0200 Subject: [PATCH 05/38] remove replica response from TransportShardReplicationOperation. not needed anymore --- .../action/bulk/TransportShardBulkAction.java | 11 +- .../action/delete/TransportDeleteAction.java | 11 +- .../TransportDeleteByQueryAction.java | 3 +- .../TransportIndexDeleteByQueryAction.java | 3 +- .../TransportShardDeleteByQueryAction.java | 11 +- .../action/index/TransportIndexAction.java | 15 +-- .../TransportBroadcastOperationAction.java | 7 +- ...nsportIndexReplicationOperationAction.java | 7 +- ...portIndicesReplicationOperationAction.java | 7 +- ...nsportShardReplicationOperationAction.java | 115 ++++++------------ 10 files changed, 56 insertions(+), 134 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 16455b0d1be..245d7d16033 100644 --- a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -60,7 +60,6 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.river.RiverIndexName; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -68,7 +67,7 @@ import java.util.Map; /** * Performs the index operation. */ -public class TransportShardBulkAction extends TransportShardReplicationOperationAction { +public class TransportShardBulkAction extends TransportShardReplicationOperationAction { private final static String OP_TYPE_UPDATE = "update"; private final static String OP_TYPE_DELETE = "delete"; @@ -119,11 +118,6 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation return new BulkShardResponse(); } - @Override - protected TransportResponse.Empty newReplicaResponseInstance() { - return TransportResponse.Empty.INSTANCE; - } - @Override protected boolean resolveIndex() { return false; @@ -534,7 +528,7 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation @Override - protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); final BulkShardRequest request = shardRequest.request; for (int i = 0; i < request.items().length; i++) { @@ -586,7 +580,6 @@ public class TransportShardBulkAction extends TransportShardReplicationOperation // ignore } } - return newReplicaResponseInstance(); } private void applyVersion(BulkItemRequest item, long version, VersionType versionType) { diff --git a/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index efdea2cb3b5..86ffac8bcce 100644 --- a/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -43,13 +43,12 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** * Performs the delete operation. */ -public class TransportDeleteAction extends TransportShardReplicationOperationAction { +public class TransportDeleteAction extends TransportShardReplicationOperationAction { private final AutoCreateIndex autoCreateIndex; @@ -141,11 +140,6 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct return new DeleteResponse(); } - @Override - protected TransportResponse.Empty newReplicaResponseInstance() { - return TransportResponse.Empty.INSTANCE; - } - @Override protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { DeleteRequest request = shardRequest.request; @@ -171,7 +165,7 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct } @Override - protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { DeleteRequest request = shardRequest.request; IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA); @@ -185,7 +179,6 @@ public class TransportDeleteAction extends TransportShardReplicationOperationAct // ignore } } - return newReplicaResponseInstance(); } @Override diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java index 2571eecef5f..0800a639a81 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryAction.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -41,7 +40,7 @@ import java.util.concurrent.atomic.AtomicReferenceArray; /** */ -public class TransportDeleteByQueryAction extends TransportIndicesReplicationOperationAction { +public class TransportDeleteByQueryAction extends TransportIndicesReplicationOperationAction { private final DestructiveOperations destructiveOperations; diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java index a76a0ee9e23..607459e7798 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportIndexDeleteByQueryAction.java @@ -30,14 +30,13 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import java.util.List; /** * Internal transport action that broadcasts a delete by query request to all of the shards that belong to an index. */ -public class TransportIndexDeleteByQueryAction extends TransportIndexReplicationOperationAction { +public class TransportIndexDeleteByQueryAction extends TransportIndexReplicationOperationAction { private static final String ACTION_NAME = DeleteByQueryAction.NAME + "[index]"; diff --git a/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java b/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java index aeb852e967e..6e364302e83 100644 --- a/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java +++ b/src/main/java/org/elasticsearch/action/deletebyquery/TransportShardDeleteByQueryAction.java @@ -42,13 +42,12 @@ import org.elasticsearch.search.internal.DefaultSearchContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchLocalRequest; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** * */ -public class TransportShardDeleteByQueryAction extends TransportShardReplicationOperationAction { +public class TransportShardDeleteByQueryAction extends TransportShardReplicationOperationAction { public final static String DELETE_BY_QUERY_API = "delete_by_query"; @@ -94,11 +93,6 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication return new ShardDeleteByQueryResponse(); } - @Override - protected TransportResponse.Empty newReplicaResponseInstance() { - return TransportResponse.Empty.INSTANCE; - } - @Override protected boolean resolveIndex() { return false; @@ -127,7 +121,7 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication @Override - protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { ShardDeleteByQueryRequest request = shardRequest.request; IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); IndexShard indexShard = indexService.shardSafe(shardRequest.shardId.id()); @@ -144,7 +138,6 @@ public class TransportShardDeleteByQueryAction extends TransportShardReplication SearchContext.removeCurrent(); } } - return newReplicaResponseInstance(); } @Override diff --git a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index e165253f0b0..79ea496c317 100644 --- a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -49,7 +49,6 @@ import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.river.RiverIndexName; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; /** @@ -62,7 +61,7 @@ import org.elasticsearch.transport.TransportService; *
  • allowIdGeneration: If the id is set not, should it be generated. Defaults to true. * */ -public class TransportIndexAction extends TransportShardReplicationOperationAction { +public class TransportIndexAction extends TransportShardReplicationOperationAction { private final AutoCreateIndex autoCreateIndex; @@ -157,11 +156,6 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi return new IndexResponse(); } - @Override - protected TransportResponse.Empty newReplicaResponseInstance() { - return TransportResponse.Empty.INSTANCE; - } - @Override protected String executor() { return ThreadPool.Names.INDEX; @@ -245,7 +239,7 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi } @Override - protected TransportResponse.Empty shardOperationOnReplica(ReplicaOperationRequest shardRequest) { + protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).shardSafe(shardRequest.shardId.id()); IndexRequest request = shardRequest.request; SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).type(request.type()).id(request.id()) @@ -265,10 +259,5 @@ public class TransportIndexAction extends TransportShardReplicationOperationActi // ignore } } - return newReplicaResponseInstance(); - } - - public String getReplicaActionName() { - return IndexAction.NAME + "[r]"; } } diff --git a/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java b/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java index 81676226f90..09d7bd55447 100644 --- a/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastOperationAction.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; @@ -59,16 +60,12 @@ public abstract class TransportBroadcastOperationAction listener) { new AsyncBroadcastAction(request, listener).start(); diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java index c4dddc7e361..53c4984bfc8 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportIndexReplicationOperationAction.java @@ -36,7 +36,6 @@ import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.TransportResponse; import java.util.ArrayList; import java.util.Arrays; @@ -49,15 +48,15 @@ import java.util.concurrent.atomic.AtomicReferenceArray; * It relies on a shard sub-action that gets sent over the transport and executed on each of the shard. * The index provided with the request is expected to be a concrete index, properly resolved by the callers (parent actions). */ -public abstract class TransportIndexReplicationOperationAction +public abstract class TransportIndexReplicationOperationAction extends TransportAction { protected final ClusterService clusterService; - protected final TransportShardReplicationOperationAction shardAction; + protected final TransportShardReplicationOperationAction shardAction; protected TransportIndexReplicationOperationAction(Settings settings, String actionName, ClusterService clusterService, - ThreadPool threadPool, TransportShardReplicationOperationAction shardAction, ActionFilters actionFilters) { + ThreadPool threadPool, TransportShardReplicationOperationAction shardAction, ActionFilters actionFilters) { super(settings, actionName, threadPool, actionFilters); this.clusterService = clusterService; this.shardAction = shardAction; diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java index c53b9bbb65d..e2a811202c1 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportIndicesReplicationOperationAction.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BaseTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.util.Map; @@ -43,15 +42,15 @@ import java.util.concurrent.atomic.AtomicReferenceArray; /** */ public abstract class TransportIndicesReplicationOperationAction + ShardRequest extends ShardReplicationOperationRequest, ShardResponse extends ActionWriteResponse> extends TransportAction { protected final ClusterService clusterService; - protected final TransportIndexReplicationOperationAction indexAction; + protected final TransportIndexReplicationOperationAction indexAction; protected TransportIndicesReplicationOperationAction(Settings settings, String actionName, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, - TransportIndexReplicationOperationAction indexAction, ActionFilters actionFilters) { + TransportIndexReplicationOperationAction indexAction, ActionFilters actionFilters) { super(settings, actionName, threadPool, actionFilters); this.clusterService = clusterService; this.indexAction = indexAction; diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java index 244d624cb15..df99d045177 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java @@ -59,14 +59,12 @@ import org.elasticsearch.transport.*; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReferenceArray; /** */ -public abstract class TransportShardReplicationOperationAction extends TransportAction { +public abstract class TransportShardReplicationOperationAction extends TransportAction { protected final TransportService transportService; protected final ClusterService clusterService; @@ -88,22 +86,18 @@ public abstract class TransportShardReplicationOperationAction listener) { new AsyncShardOperationAction(request, listener).start(); @@ -115,17 +109,15 @@ public abstract class TransportShardReplicationOperationAction shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable; + protected abstract Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable; - protected abstract ReplicaResponse shardOperationOnReplica(ReplicaOperationRequest shardRequest); + protected abstract void shardOperationOnReplica(ReplicaOperationRequest shardRequest); protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException; @@ -224,7 +216,7 @@ public abstract class TransportShardReplicationOperationAction { + class ReplicaOperationTransportHandler extends BaseTransportRequestHandler { @Override public ReplicaOperationRequest newInstance() { @@ -244,14 +236,13 @@ public abstract class TransportShardReplicationOperationAction() { - @Override - public ReplicaResponse newInstance() { - return newReplicaResponseInstance(); - } + transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { + @Override + public void handleResponse(TransportResponse.Empty vResponse) { + state.onReplicaSuccess(); + } - @Override - public void handleResponse(ReplicaResponse vResponse) { - state.onReplicaSuccess(vResponse); - } + @Override + public void handleException(TransportException exp) { + state.onReplicaFailure(nodeId, exp); + logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request()); + if (!ignoreReplicaException(exp)) { + logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp); + shardStateAction.shardFailed(shard, indexMetaData.getUUID(), + "Failed to perform [" + actionName + "] on replica, message [" + ExceptionsHelper.detailedMessage(exp) + "]"); + } + } - @Override - public void handleException(TransportException exp) { - state.onReplicaFailure(nodeId, exp); - logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request()); - if (!ignoreReplicaException(exp)) { - logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp); - shardStateAction.shardFailed(shard, indexMetaData.getUUID(), - "Failed to perform [" + actionName + "] on replica, message [" + ExceptionsHelper.detailedMessage(exp) + "]"); - } - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - - }); + }); } else { if (internalRequest.request().operationThreaded()) { try { @@ -687,8 +655,8 @@ public abstract class TransportShardReplicationOperationAction 2) { // only for more than 2 in the number of shardIt it makes sense, otherwise its 1 shard with 1 replica, quorum is 1 (which is what it is initialized to) requiredNumber = (shardRoutingTable.getSize() / 2) + 1; @@ -770,7 +738,7 @@ public abstract class TransportShardReplicationOperationAction shardReplicaFailures = ConcurrentCollections.newConcurrentMap(); - // nocommit the Broadcast operations use AtomicReferencArray, Boaz wants to figure out why, this here is just a hack - private final CopyOnWriteArrayList replicaResponses = new CopyOnWriteArrayList<>(); private final AtomicInteger pending; private final int numberOfShardInstances; @@ -834,9 +800,8 @@ public abstract class TransportShardReplicationOperationAction replicaResponses) { - return finalResponse; - } - /** * Internal request class that gets built on each node. Holds the original request plus additional info. */ From 43eae13c5702033327afe953adc078880ecbf1f8 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Fri, 24 Apr 2015 10:57:53 +0200 Subject: [PATCH 06/38] use sync id when recovering Skip phase 1 of recovery in case an identical sync id was found on primary and replica. Relates to #10032 closes #10775 --- .../gateway/GatewayAllocator.java | 30 ++-- .../index/engine/InternalEngine.java | 39 ++++- .../BlobStoreIndexShardRepository.java | 22 +-- .../org/elasticsearch/index/store/Store.java | 138 +++++++++------ .../recovery/RecoveryCleanFilesRequest.java | 2 +- .../recovery/RecoverySourceHandler.java | 85 ++++++---- .../indices/recovery/RecoveryState.java | 69 ++++++-- .../indices/recovery/RecoveryTarget.java | 9 +- .../recovery/StartRecoveryRequest.java | 36 ++-- .../TransportNodesListShardStoreMetaData.java | 40 +++-- .../gateway/RecoveryFromGatewayTests.java | 159 ++++++++++++++---- .../index/engine/InternalEngineTests.java | 2 +- .../elasticsearch/index/store/StoreTest.java | 159 +++++++++++++++--- .../recovery/StartRecoveryRequestTest.java | 10 +- 14 files changed, 558 insertions(+), 242 deletions(-) diff --git a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index 399e9607cfb..84d63ed3322 100644 --- a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -330,17 +330,27 @@ public class GatewayAllocator extends AbstractComponent { if (primaryNodeStore != null && primaryNodeStore.allocated()) { long sizeMatched = 0; - for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) { - if (primaryNodeStore.fileExists(storeFileMetaData.name()) && primaryNodeStore.file(storeFileMetaData.name()).isSame(storeFileMetaData)) { - sizeMatched += storeFileMetaData.length(); - } - } - logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data", - shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched); - if (sizeMatched > lastSizeMatched) { - lastSizeMatched = sizeMatched; - lastDiscoNodeMatched = discoNode; + // see if we have a sync id we can make use of + if (storeFilesMetaData.syncId() != null && storeFilesMetaData.syncId().equals(primaryNodeStore.syncId())) { + logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), storeFilesMetaData.syncId()); lastNodeMatched = node; + lastSizeMatched = Long.MAX_VALUE; + lastDiscoNodeMatched = discoNode; + } else { + for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) { + logger.trace("{}: node [{}] has file {}", + shard, discoNode.name(), storeFileMetaData.name()); + if (primaryNodeStore.fileExists(storeFileMetaData.name()) && primaryNodeStore.file(storeFileMetaData.name()).isSame(storeFileMetaData)) { + sizeMatched += storeFileMetaData.length(); + } + } + logger.trace("{}: node [{}] has [{}/{}] bytes of re-usable data", + shard, discoNode.name(), new ByteSizeValue(sizeMatched), sizeMatched); + if (sizeMatched > lastSizeMatched) { + lastSizeMatched = sizeMatched; + lastDiscoNodeMatched = discoNode; + lastNodeMatched = node; + } } } } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1a717416a87..d038509c77e 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -73,7 +73,9 @@ public class InternalEngine extends Engine { private final FailEngineOnMergeFailure mergeSchedulerFailureListener; private final MergeSchedulerListener mergeSchedulerListener; - /** When we last pruned expired tombstones from versionMap.deletes: */ + /** + * When we last pruned expired tombstones from versionMap.deletes: + */ private volatile long lastDeleteVersionPruneTimeMSec; private final ShardIndexingService indexingService; @@ -150,10 +152,21 @@ public class InternalEngine extends Engine { long nextTranslogID = translogId.v2(); translog.newTranslog(nextTranslogID); translogIdGenerator.set(nextTranslogID); + if (translogId.v1() != null && skipInitialTranslogRecovery == false) { + // recovering from local store recoverFromTranslog(translogId.v1(), transformer); } else { - flush(true, true); + // recovering from a different source + // nocommit + // when we create the Engine on a target shard after recovery we must make sure that + // if a sync id is there then it is not overwritten by a forced flush + if (lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID) == null) { + flush(true, true); + } else { + SyncedFlushResult syncedFlushResult = syncFlushIfNoPendingChanges(lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID), lastCommittedSegmentInfos.getId()); + assert syncedFlushResult.equals(SyncedFlushResult.SUCCESS) : "skipped translog recovery but synced flush failed"; + } } } catch (IOException | EngineException ex) { throw new EngineCreationFailureException(shardId, "failed to recover from translog", ex); @@ -185,7 +198,7 @@ public class InternalEngine extends Engine { final long currentTranslogId = Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY)); return new Tuple<>(currentTranslogId, nextTranslogId); } - // translog id is not in the metadata - fix this inconsistency some code relies on this and old indices might not have it. + // translog id is not in the metadata - fix this inconsistency some code relies on this and old indices might not have it. writer.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(nextTranslogId))); commitIndexWriter(writer); logger.debug("no translog ID present in the current commit - creating one"); @@ -1058,7 +1071,8 @@ public class InternalEngine extends Engine { boolean verbose = false; try { verbose = Boolean.parseBoolean(System.getProperty("tests.verbose")); - } catch (Throwable ignore) {} + } catch (Throwable ignore) { + } iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); iwc.setMergeScheduler(mergeScheduler.newMergeScheduler()); MergePolicy mergePolicy = mergePolicyProvider.getMergePolicy(); @@ -1109,7 +1123,9 @@ public class InternalEngine extends Engine { } } - /** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */ + /** + * Extended SearcherFactory that warms the segments if needed when acquiring a new searcher + */ class SearchFactory extends EngineSearcherFactory { SearchFactory(EngineConfig engineConfig) { @@ -1271,9 +1287,20 @@ public class InternalEngine extends Engine { IOUtils.closeWhileHandlingException(translog); throw new EngineException(shardId, "failed to recover from translog", e); } - flush(true, true); + + // nocommit: when we recover from gateway we recover ops from the translog we found and then create a new translog with new id. + // we flush here because we need to write a new translog id after recovery. + // we need to make sure here that an existing sync id is not overwritten by this flush if one exists. + // so, in case the old translog did not contain any ops, we should use the old sync id for flushing. + // nocommit because not sure if this here is the best solution for this... if (operationsRecovered > 0) { + flush(true, true); refresh("translog recovery"); + } else if (lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID) == null) { + flush(true, true); + } else { + SyncedFlushResult syncedFlushResult = syncFlushIfNoPendingChanges(lastCommittedSegmentInfos.getUserData().get(SYNC_COMMIT_ID), lastCommittedSegmentInfos.getId()); + assert syncedFlushResult.equals(SyncedFlushResult.SUCCESS) : "no operations during translog recovery but synced flush failed"; } translog.clearUnreferenced(); } diff --git a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java index d8c906886b0..7e9360ccbbc 100644 --- a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java +++ b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java @@ -186,7 +186,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements @Override public void verify(String seed) { - BlobContainer testBlobContainer = blobStore.blobContainer(basePath);; + BlobContainer testBlobContainer = blobStore.blobContainer(basePath); DiscoveryNode localNode = clusterService.localNode(); if (testBlobContainer.blobExists(testBlobPrefix(seed) + "-master")) { try (OutputStream outputStream = testBlobContainer.createOutput(testBlobPrefix(seed) + "-" + localNode.getId())) { @@ -232,7 +232,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements * Serializes snapshot to JSON * * @param snapshot snapshot - * @param stream the stream to output the snapshot JSON represetation to + * @param stream the stream to output the snapshot JSON represetation to * @throws IOException if an IOException occurs */ public static void writeSnapshot(BlobStoreIndexShardSnapshot snapshot, OutputStream stream) throws IOException { @@ -247,7 +247,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements * @param stream JSON * @return snapshot * @throws IOException if an IOException occurs - * */ + */ public static BlobStoreIndexShardSnapshot readSnapshot(InputStream stream) throws IOException { try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(stream)) { parser.nextToken(); @@ -314,7 +314,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements public BlobStoreIndexShardSnapshot loadSnapshot() { BlobStoreIndexShardSnapshot snapshot; try (InputStream stream = blobContainer.openInput(snapshotBlobName(snapshotId))) { - snapshot = readSnapshot(stream); + snapshot = readSnapshot(stream); } catch (IOException ex) { throw new IndexShardRestoreFailedException(shardId, "failed to read shard snapshot file", ex); } @@ -472,7 +472,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements // don't have this hash we try to read that hash from the blob store // in a bwc compatible way. maybeRecalculateMetadataHash(blobContainer, fileInfo, metadata); - } catch (Throwable e) { + } catch (Throwable e) { logger.warn("{} Can't calculate hash from blob for file [{}] [{}]", e, shardId, fileInfo.physicalName(), fileInfo.metadata()); } if (fileInfo == null || !fileInfo.isSame(md) || !snapshotFileExistsInBlobs(fileInfo, blobs)) { @@ -550,7 +550,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) { for (int i = 0; i < fileInfo.numberOfParts(); i++) { final InputStreamIndexInput inputStreamIndexInput = new InputStreamIndexInput(indexInput, fileInfo.partBytes()); - InputStream inputStream = snapshotRateLimiter == null ? inputStreamIndexInput : new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, snapshotThrottleListener); + InputStream inputStream = snapshotRateLimiter == null ? inputStreamIndexInput : new RateLimitingInputStream(inputStreamIndexInput, snapshotRateLimiter, snapshotThrottleListener); inputStream = new AbortableInputStream(inputStream, fileInfo.physicalName()); try (OutputStream output = blobContainer.createOutput(fileInfo.partName(i))) { int len; @@ -727,14 +727,14 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements // don't have this hash we try to read that hash from the blob store // in a bwc compatible way. maybeRecalculateMetadataHash(blobContainer, fileInfo, recoveryTargetMetadata); - } catch (Throwable e) { + } catch (Throwable e) { // if the index is broken we might not be able to read it logger.warn("{} Can't calculate hash from blog for file [{}] [{}]", e, shardId, fileInfo.physicalName(), fileInfo.metadata()); } snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); fileInfos.put(fileInfo.metadata().name(), fileInfo); } - final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData); + final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData, Collections.EMPTY_MAP); final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); for (StoreFileMetaData md : diff.identical) { FileInfo fileInfo = fileInfos.get(md.name()); @@ -804,8 +804,8 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements try (final IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { final byte[] buffer = new byte[BUFFER_SIZE]; int length; - while((length=stream.read(buffer))>0){ - indexOutput.writeBytes(buffer,0,length); + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, 0, length); recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); if (restoreRateLimiter != null) { rateLimiterListener.onRestorePause(restoreRateLimiter.pause(length)); @@ -838,7 +838,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements } } - + public interface RateLimiterListener { void onRestorePause(long nanos); diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index 71dd77c690e..de86d3b4dc3 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -28,6 +28,7 @@ import org.apache.lucene.store.*; import org.apache.lucene.util.*; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.apache.lucene.util.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; @@ -35,6 +36,7 @@ import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; @@ -46,6 +48,7 @@ import org.elasticsearch.common.util.SingleObjectCache; import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.RefCounted; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.ShardId; @@ -166,10 +169,10 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * Returns a new MetadataSnapshot for the latest commit in this store or * an empty snapshot if no index exists or can not be opened. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. */ public MetadataSnapshot getMetadataOrEmpty() throws IOException { try { @@ -185,13 +188,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref /** * Returns a new MetadataSnapshot for the latest commit in this store. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. - * @throws FileNotFoundException if one or more files referenced by a commit are not present. - * @throws NoSuchFileException if one or more files referenced by a commit are not present. - * @throws IndexNotFoundException if no index / valid commit-point can be found in this store + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws FileNotFoundException if one or more files referenced by a commit are not present. + * @throws NoSuchFileException if one or more files referenced by a commit are not present. + * @throws IndexNotFoundException if no index / valid commit-point can be found in this store */ public MetadataSnapshot getMetadata() throws IOException { return getMetadata(null); @@ -201,13 +204,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * Returns a new MetadataSnapshot for the given commit. If the given commit is null * the latest commit point is used. * - * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an - * unexpected exception when opening the index reading the segments file. - * @throws IndexFormatTooOldException if the lucene index is too old to be opened. - * @throws IndexFormatTooNewException if the lucene index is too new to be opened. - * @throws FileNotFoundException if one or more files referenced by a commit are not present. - * @throws NoSuchFileException if one or more files referenced by a commit are not present. - * @throws IndexNotFoundException if the commit point can't be found in this store + * @throws CorruptIndexException if the lucene index is corrupted. This can be caused by a checksum mismatch or an + * unexpected exception when opening the index reading the segments file. + * @throws IndexFormatTooOldException if the lucene index is too old to be opened. + * @throws IndexFormatTooNewException if the lucene index is too new to be opened. + * @throws FileNotFoundException if one or more files referenced by a commit are not present. + * @throws NoSuchFileException if one or more files referenced by a commit are not present. + * @throws IndexNotFoundException if the commit point can't be found in this store */ public MetadataSnapshot getMetadata(IndexCommit commit) throws IOException { ensureOpen(); @@ -363,7 +366,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * @throws IOException if the index we try to read is corrupted */ public static MetadataSnapshot readMetadataSnapshot(Path indexLocation, ESLogger logger) throws IOException { - try (Directory dir = new SimpleFSDirectory(indexLocation)){ + try (Directory dir = new SimpleFSDirectory(indexLocation)) { failIfCorrupted(dir, new ShardId("", 1)); return new MetadataSnapshot(null, dir, logger); } catch (IndexNotFoundException ex) { @@ -433,7 +436,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref public boolean checkIntegrityNoException(StoreFileMetaData md) { return checkIntegrityNoException(md, directory()); } - + public static boolean checkIntegrityNoException(StoreFileMetaData md, Directory directory) { try { checkIntegrity(md, directory); @@ -454,7 +457,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref // throw exception if metadata is inconsistent if (!checksum.equals(md.checksum())) { throw new CorruptIndexException("inconsistent metadata: lucene checksum=" + checksum + - ", metadata checksum=" + md.checksum(), input); + ", metadata checksum=" + md.checksum(), input); } } else if (md.hasLegacyChecksum()) { // legacy checksum verification - no footer that we need to omit in the checksum! @@ -472,7 +475,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref String adler32 = Store.digestToString(checksum.getValue()); if (!adler32.equals(md.checksum())) { throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + md.checksum() + - " actual=" + adler32, input); + " actual=" + adler32, input); } } } @@ -530,7 +533,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * * @param reason the reason for this cleanup operation logged for each deleted file * @param sourceMetaData the metadata used for cleanup. all files in this metadata should be kept around. - * @throws IOException if an IOException occurs + * @throws IOException if an IOException occurs * @throws IllegalStateException if the latest snapshot in this store differs from the given one after the cleanup. */ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData) throws IOException { @@ -549,7 +552,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref // FNF should not happen since we hold a write lock? } catch (IOException ex) { if (existingFile.startsWith(IndexFileNames.SEGMENTS) - || existingFile.equals(IndexFileNames.OLD_SEGMENTS_GEN)) { + || existingFile.equals(IndexFileNames.OLD_SEGMENTS_GEN)) { // TODO do we need to also fail this if we can't delete the pending commit file? // if one of those files can't be deleted we better fail the cleanup otherwise we might leave an old commit point around? throw new IllegalStateException("Can't delete " + existingFile + " - cleanup failed", ex); @@ -656,32 +659,60 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref * * @see StoreFileMetaData */ - public final static class MetadataSnapshot implements Iterable, Streamable { + public final static class MetadataSnapshot implements Iterable, Writeable { private static final ESLogger logger = Loggers.getLogger(MetadataSnapshot.class); private static final Version FIRST_LUCENE_CHECKSUM_VERSION = Version.LUCENE_4_8; - private Map metadata; + private final ImmutableMap metadata; public static final MetadataSnapshot EMPTY = new MetadataSnapshot(); - public MetadataSnapshot(Map metadata) { - this.metadata = metadata; + private final ImmutableMap commitUserData; + + public MetadataSnapshot(Map metadata, Map commitUserData) { + ImmutableMap.Builder metaDataBuilder = ImmutableMap.builder(); + this.metadata = metaDataBuilder.putAll(metadata).build(); + ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); + this.commitUserData = commitUserDataBuilder.putAll(commitUserData).build(); } MetadataSnapshot() { - this.metadata = Collections.emptyMap(); + metadata = ImmutableMap.of(); + commitUserData = ImmutableMap.of(); } MetadataSnapshot(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { - metadata = buildMetadata(commit, directory, logger); + Tuple, ImmutableMap> loadedMetadata = loadMetadata(commit, directory, logger); + metadata = loadedMetadata.v1(); + commitUserData = loadedMetadata.v2(); assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); } - ImmutableMap buildMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { + public MetadataSnapshot(StreamInput in) throws IOException { + int size = in.readVInt(); + ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); + for (int i = 0; i < size; i++) { + StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in); + metadataBuilder.put(meta.name(), meta); + } + ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); + int num = in.readVInt(); + for (int i = num; i > 0; i--) { + commitUserDataBuilder.put(in.readString(), in.readString()); + } + + this.commitUserData = commitUserDataBuilder.build(); + this.metadata = metadataBuilder.build(); + assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); + } + + static Tuple, ImmutableMap> loadMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { ImmutableMap.Builder builder = ImmutableMap.builder(); Map checksumMap = readLegacyChecksums(directory).v1(); + ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); try { final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory); + commitUserDataBuilder.putAll(segmentCommitInfos.getUserData()); Version maxVersion = Version.LUCENE_4_0; // we don't know which version was used to write so we take the max version. for (SegmentCommitInfo info : segmentCommitInfos) { final Version version = info.info.getVersion(); @@ -734,7 +765,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throw ex; } - return builder.build(); + return new Tuple, ImmutableMap>(builder.build(), commitUserDataBuilder.build()); } /** @@ -955,30 +986,21 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref return metadata.size(); } - public static MetadataSnapshot read(StreamInput in) throws IOException { - MetadataSnapshot storeFileMetaDatas = new MetadataSnapshot(); - storeFileMetaDatas.readFrom(in); - return storeFileMetaDatas; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - int size = in.readVInt(); - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (int i = 0; i < size; i++) { - StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in); - builder.put(meta.name(), meta); - } - this.metadata = builder.build(); - assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(this.metadata.size()); for (StoreFileMetaData meta : this) { meta.writeTo(out); } + out.writeVInt(commitUserData.size()); + for (Map.Entry entry : commitUserData.entrySet()) { + out.writeString(entry.getKey()); + out.writeString(entry.getValue()); + } + } + + public Map getCommitUserData() { + return commitUserData; } /** @@ -1010,6 +1032,20 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } return count; } + + /** + * Returns the sync id of the commit point that this MetadataSnapshot represents. + * + * @return sync id if exists, else null + */ + public String getSyncId() { + return commitUserData.get(Engine.SYNC_COMMIT_ID); + } + + @Override + public MetadataSnapshot readFrom(StreamInput in) throws IOException { + return new MetadataSnapshot(in); + } } /** @@ -1360,7 +1396,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } @Override - protected StoreStats refresh() { + protected StoreStats refresh() { try { return new StoreStats(estimateSize(directory), directoryService.throttleTimeInNanos()); } catch (IOException ex) { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java index 0ff00d7c008..b0d224c41ef 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryCleanFilesRequest.java @@ -61,7 +61,7 @@ class RecoveryCleanFilesRequest extends TransportRequest { super.readFrom(in); recoveryId = in.readLong(); shardId = ShardId.readShardId(in); - snapshotFiles = Store.MetadataSnapshot.read(in); + snapshotFiles = new Store.MetadataSnapshot(in); totalTranslogOps = in.readVInt(); } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 6a429974f69..2535caa7c0b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -65,6 +65,7 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.concurrent.*; @@ -137,11 +138,11 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { * Perform phase1 of the recovery operations. Once this {@link SnapshotIndexCommit} * snapshot has been performed no commit operations (files being fsync'd) * are effectively allowed on this index until all recovery phases are done - * + *

    * Phase1 examines the segment files on the target node and copies over the * segments that are missing. Only segments that have the same size and * checksum can be reused - * + *

    * {@code InternalEngine#recover} is responsible for snapshotting the index * and releasing the snapshot once all 3 phases of recovery are complete */ @@ -168,28 +169,45 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { // Generate a "diff" of all the identical, different, and missing // segment files on the target node, using the existing files on // the source node - final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(new Store.MetadataSnapshot(request.existingFiles())); - for (StoreFileMetaData md : diff.identical) { - response.phase1ExistingFileNames.add(md.name()); - response.phase1ExistingFileSizes.add(md.length()); - existingTotalSize += md.length(); - if (logger.isTraceEnabled()) { - logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}], size [{}]", - indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length()); + String recoverySourceSyncId = recoverySourceMetadata.getSyncId(); + String recoveryTargetSyncId = request.metadataSnapshot().getSyncId(); + final boolean recoverWithSyncId = recoverySourceSyncId != null && + recoverySourceSyncId.equals(recoveryTargetSyncId); + if (recoverWithSyncId) { + for (StoreFileMetaData md : request.metadataSnapshot()) { + response.phase1ExistingFileNames.add(md.name()); + response.phase1ExistingFileSizes.add(md.length()); + existingTotalSize += md.length(); + if (logger.isTraceEnabled()) { + logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], checksum [{}], size [{}], sync ids {} coincide, will skip file copy", + indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length(), recoverySourceMetadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID)); + } + totalSize += md.length(); } - totalSize += md.length(); - } - for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { - if (request.existingFiles().containsKey(md.name())) { - logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote [{}], local [{}]", - indexName, shardId, request.targetNode(), md.name(), request.existingFiles().get(md.name()), md); - } else { - logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote", - indexName, shardId, request.targetNode(), md.name()); + } else { + final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(request.metadataSnapshot()); + for (StoreFileMetaData md : diff.identical) { + response.phase1ExistingFileNames.add(md.name()); + response.phase1ExistingFileSizes.add(md.length()); + existingTotalSize += md.length(); + if (logger.isTraceEnabled()) { + logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}], size [{}]", + indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length()); + } + totalSize += md.length(); + } + for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { + if (request.metadataSnapshot().asMap().containsKey(md.name())) { + logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], exists in local store, but is different: remote [{}], local [{}]", + indexName, shardId, request.targetNode(), md.name(), request.metadataSnapshot().asMap().get(md.name()), md); + } else { + logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote", + indexName, shardId, request.targetNode(), md.name()); + } + response.phase1FileNames.add(md.name()); + response.phase1FileSizes.add(md.length()); + totalSize += md.length(); } - response.phase1FileNames.add(md.name()); - response.phase1FileSizes.add(md.length()); - totalSize += md.length(); } response.phase1TotalSize = totalSize; response.phase1ExistingTotalSize = existingTotalSize; @@ -209,7 +227,6 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { } }); - // This latch will be used to wait until all files have been transferred to the target node final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size()); final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); @@ -364,8 +381,9 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { // related to this recovery (out of date segments, for example) // are deleted try { + final Store.MetadataSnapshot remainingFilesAfterCleanup = recoverWithSyncId? request.metadataSnapshot(): recoverySourceMetadata; transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, - new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, shard.translog().estimatedNumberOfOperations()), + new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), remainingFilesAfterCleanup, shard.translog().estimatedNumberOfOperations()), TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); } catch (RemoteTransportException remoteException) { @@ -418,12 +436,12 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { /** * Perform phase2 of the recovery process - * + *

    * Phase2 takes a snapshot of the current translog *without* acquiring the * write lock (however, the translog snapshot is a point-in-time view of * the translog). It then sends each translog operation to the target node * so it can be replayed into the new shard. - * + *

    * {@code InternalEngine#recover} is responsible for taking the snapshot * of the translog and releasing it once all 3 phases of recovery are complete */ @@ -469,11 +487,11 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { /** * Perform phase 3 of the recovery process - * + *

    * Phase3 again takes a snapshot of the translog, however this time the * snapshot is acquired under a write lock. The translog operations are * sent to the target node where they are replayed. - * + *

    * {@code InternalEngine#recover} is responsible for taking the snapshot * of the translog, and after phase 3 completes the snapshots from all * three phases are released. @@ -587,7 +605,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { /** * Send the given snapshot's operations to this handler's target node. - * + *

    * Operations are bulked into a single request depending on an operation * count limit or size-in-bytes limit * @@ -600,8 +618,8 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { final List operations = Lists.newArrayList(); Translog.Operation operation; try { - operation = snapshot.next(); // this ex should bubble up - } catch (IOException ex){ + operation = snapshot.next(); // this ex should bubble up + } catch (IOException ex) { throw new ElasticsearchException("failed to get next operation from translog", ex); } @@ -659,9 +677,10 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler { } try { operation = snapshot.next(); // this ex should bubble up - } catch (IOException ex){ + } catch (IOException ex) { throw new ElasticsearchException("failed to get next operation from translog", ex); - } } + } + } // send the leftover if (!operations.isEmpty()) { cancellableThreads.execute(new Interruptable() { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index c2af0657bc5..2f8339afc21 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.logging.ESLoggerFactory; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -47,16 +48,24 @@ public class RecoveryState implements ToXContent, Streamable { public static enum Stage { INIT((byte) 0), - /** recovery of lucene files, either reusing local ones are copying new ones */ + /** + * recovery of lucene files, either reusing local ones are copying new ones + */ INDEX((byte) 1), - /** potentially running check index */ + /** + * potentially running check index + */ VERIFY_INDEX((byte) 2), - /** starting up the engine, replaying the translog */ + /** + * starting up the engine, replaying the translog + */ TRANSLOG((byte) 3), - /** performing final task after all translog ops have been done */ + /** + * performing final task after all translog ops have been done + */ FINALIZE((byte) 4), DONE((byte) 5); @@ -494,7 +503,9 @@ public class RecoveryState implements ToXContent, Streamable { assert total == UNKNOWN || total >= recovered : "total, if known, should be > recovered. total [" + total + "], recovered [" + recovered + "]"; } - /** returns the total number of translog operations recovered so far */ + /** + * returns the total number of translog operations recovered so far + */ public synchronized int recoveredOperations() { return recovered; } @@ -587,22 +598,30 @@ public class RecoveryState implements ToXContent, Streamable { recovered += bytes; } - /** file name * */ + /** + * file name * + */ public String name() { return name; } - /** file length * */ + /** + * file length * + */ public long length() { return length; } - /** number of bytes recovered for this file (so far). 0 if the file is reused * */ + /** + * number of bytes recovered for this file (so far). 0 if the file is reused * + */ public long recovered() { return recovered; } - /** returns true if the file is reused from a local copy */ + /** + * returns true if the file is reused from a local copy + */ public boolean reused() { return reused; } @@ -729,12 +748,16 @@ public class RecoveryState implements ToXContent, Streamable { return TimeValue.timeValueNanos(targetThrottleTimeInNanos); } - /** total number of files that are part of this recovery, both re-used and recovered */ + /** + * total number of files that are part of this recovery, both re-used and recovered + */ public synchronized int totalFileCount() { return fileDetails.size(); } - /** total number of files to be recovered (potentially not yet done) */ + /** + * total number of files to be recovered (potentially not yet done) + */ public synchronized int totalRecoverFiles() { int total = 0; for (File file : fileDetails.values()) { @@ -746,7 +769,9 @@ public class RecoveryState implements ToXContent, Streamable { } - /** number of file that were recovered (excluding on ongoing files) */ + /** + * number of file that were recovered (excluding on ongoing files) + */ public synchronized int recoveredFileCount() { int count = 0; for (File file : fileDetails.values()) { @@ -757,7 +782,9 @@ public class RecoveryState implements ToXContent, Streamable { return count; } - /** percent of recovered (i.e., not reused) files out of the total files to be recovered */ + /** + * percent of recovered (i.e., not reused) files out of the total files to be recovered + */ public synchronized float recoveredFilesPercent() { int total = 0; int recovered = 0; @@ -780,7 +807,9 @@ public class RecoveryState implements ToXContent, Streamable { } } - /** total number of bytes in th shard */ + /** + * total number of bytes in th shard + */ public synchronized long totalBytes() { long total = 0; for (File file : fileDetails.values()) { @@ -789,7 +818,9 @@ public class RecoveryState implements ToXContent, Streamable { return total; } - /** total number of bytes recovered so far, including both existing and reused */ + /** + * total number of bytes recovered so far, including both existing and reused + */ public synchronized long recoveredBytes() { long recovered = 0; for (File file : fileDetails.values()) { @@ -798,7 +829,9 @@ public class RecoveryState implements ToXContent, Streamable { return recovered; } - /** total bytes of files to be recovered (potentially not yet done) */ + /** + * total bytes of files to be recovered (potentially not yet done) + */ public synchronized long totalRecoverBytes() { long total = 0; for (File file : fileDetails.values()) { @@ -819,7 +852,9 @@ public class RecoveryState implements ToXContent, Streamable { return total; } - /** percent of bytes recovered out of total files bytes *to be* recovered */ + /** + * percent of bytes recovered out of total files bytes *to be* recovered + */ public synchronized float recoveredBytesPercent() { long total = 0; long recovered = 0; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index fcfc9722a03..185f1e8cd07 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.index.IndexShardMissingException; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.*; @@ -155,12 +156,12 @@ public class RecoveryTarget extends AbstractComponent { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; logger.trace("collecting local files for {}", recoveryStatus); - Map existingFiles; + Store.MetadataSnapshot metadataSnapshot = null; try { - existingFiles = recoveryStatus.store().getMetadataOrEmpty().asMap(); + metadataSnapshot = recoveryStatus.store().getMetadataOrEmpty(); } catch (IOException e) { logger.warn("error while listing local files, recover as if there are none", e); - existingFiles = Store.MetadataSnapshot.EMPTY.asMap(); + metadataSnapshot = Store.MetadataSnapshot.EMPTY; } catch (Exception e) { // this will be logged as warning later on... logger.trace("unexpected error while listing local files, failing recovery", e); @@ -169,7 +170,7 @@ public class RecoveryTarget extends AbstractComponent { return; } final StartRecoveryRequest request = new StartRecoveryRequest(recoveryStatus.shardId(), recoveryStatus.sourceNode(), clusterService.localNode(), - false, existingFiles, recoveryStatus.state().getType(), recoveryStatus.recoveryId()); + false, metadataSnapshot, recoveryStatus.state().getType(), recoveryStatus.recoveryId()); final AtomicReference responseHolder = new AtomicReference<>(); try { diff --git a/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index 88c14d7f0ce..b5fa3ee6047 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.transport.TransportRequest; @@ -46,7 +47,7 @@ public class StartRecoveryRequest extends TransportRequest { private boolean markAsRelocated; - private Map existingFiles; + private Store.MetadataSnapshot metadataSnapshot; private RecoveryState.Type recoveryType; @@ -57,20 +58,19 @@ public class StartRecoveryRequest extends TransportRequest { * Start recovery request. * * @param shardId - * @param sourceNode The node to recover from - * @param targetNode The node to recover to + * @param sourceNode The node to recover from + * @param targetNode The node to recover to * @param markAsRelocated - * @param existingFiles + * @param metadataSnapshot */ - public StartRecoveryRequest(ShardId shardId, DiscoveryNode sourceNode, DiscoveryNode targetNode, boolean markAsRelocated, Map existingFiles, RecoveryState.Type recoveryType, long recoveryId) { + public StartRecoveryRequest(ShardId shardId, DiscoveryNode sourceNode, DiscoveryNode targetNode, boolean markAsRelocated, Store.MetadataSnapshot metadataSnapshot, RecoveryState.Type recoveryType, long recoveryId) { this.recoveryId = recoveryId; this.shardId = shardId; this.sourceNode = sourceNode; this.targetNode = targetNode; this.markAsRelocated = markAsRelocated; - this.existingFiles = existingFiles; this.recoveryType = recoveryType; + this.metadataSnapshot = metadataSnapshot; } public long recoveryId() { @@ -93,14 +93,14 @@ public class StartRecoveryRequest extends TransportRequest { return markAsRelocated; } - public Map existingFiles() { - return existingFiles; - } - public RecoveryState.Type recoveryType() { return recoveryType; } + public Store.MetadataSnapshot metadataSnapshot() { + return metadataSnapshot; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -109,13 +109,9 @@ public class StartRecoveryRequest extends TransportRequest { sourceNode = DiscoveryNode.readNode(in); targetNode = DiscoveryNode.readNode(in); markAsRelocated = in.readBoolean(); - int size = in.readVInt(); - existingFiles = Maps.newHashMapWithExpectedSize(size); - for (int i = 0; i < size; i++) { - StoreFileMetaData md = StoreFileMetaData.readStoreFileMetaData(in); - existingFiles.put(md.name(), md); - } + metadataSnapshot = new Store.MetadataSnapshot(in); recoveryType = RecoveryState.Type.fromId(in.readByte()); + } @Override @@ -126,10 +122,8 @@ public class StartRecoveryRequest extends TransportRequest { sourceNode.writeTo(out); targetNode.writeTo(out); out.writeBoolean(markAsRelocated); - out.writeVInt(existingFiles.size()); - for (StoreFileMetaData md : existingFiles.values()) { - md.writeTo(out); - } + metadataSnapshot.writeTo(out); out.writeByte(recoveryType.id()); } + } diff --git a/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java b/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java index f95cfd8ece9..e65a260eb06 100644 --- a/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java +++ b/src/main/java/org/elasticsearch/indices/store/TransportNodesListShardStoreMetaData.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; @@ -144,7 +145,7 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio store.incRef(); try { exists = true; - return new StoreFilesMetaData(true, shardId, store.getMetadataOrEmpty().asMap()); + return new StoreFilesMetaData(true, shardId, store.getMetadataOrEmpty()); } finally { store.decRef(); } @@ -153,17 +154,17 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio // try and see if we an list unallocated IndexMetaData metaData = clusterService.state().metaData().index(shardId.index().name()); if (metaData == null) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } String storeType = metaData.settings().get(IndexStoreModule.STORE_TYPE, "fs"); if (!storeType.contains("fs")) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, metaData.settings()); if (shardPath == null) { - return new StoreFilesMetaData(false, shardId, ImmutableMap.of()); + return new StoreFilesMetaData(false, shardId, Store.MetadataSnapshot.EMPTY); } - return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardPath.resolveIndex(), logger).asMap()); + return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardPath.resolveIndex(), logger)); } finally { TimeValue took = new TimeValue(System.currentTimeMillis() - startTime); if (exists) { @@ -180,17 +181,18 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio } public static class StoreFilesMetaData implements Iterable, Streamable { + // here also trasmit sync id, else recovery will not use sync id because of stupid gateway allocator every now and then... private boolean allocated; private ShardId shardId; - private Map files; + Store.MetadataSnapshot metadataSnapshot; StoreFilesMetaData() { } - public StoreFilesMetaData(boolean allocated, ShardId shardId, Map files) { + public StoreFilesMetaData(boolean allocated, ShardId shardId, Store.MetadataSnapshot metadataSnapshot) { this.allocated = allocated; this.shardId = shardId; - this.files = files; + this.metadataSnapshot = metadataSnapshot; } public boolean allocated() { @@ -203,15 +205,15 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio @Override public Iterator iterator() { - return files.values().iterator(); + return metadataSnapshot.iterator(); } public boolean fileExists(String name) { - return files.containsKey(name); + return metadataSnapshot.asMap().containsKey(name); } public StoreFileMetaData file(String name) { - return files.get(name); + return metadataSnapshot.asMap().get(name); } public static StoreFilesMetaData readStoreFilesMetaData(StreamInput in) throws IOException { @@ -224,22 +226,18 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio public void readFrom(StreamInput in) throws IOException { allocated = in.readBoolean(); shardId = ShardId.readShardId(in); - int size = in.readVInt(); - files = Maps.newHashMapWithExpectedSize(size); - for (int i = 0; i < size; i++) { - StoreFileMetaData md = StoreFileMetaData.readStoreFileMetaData(in); - files.put(md.name(), md); - } + this.metadataSnapshot = new Store.MetadataSnapshot(in); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(allocated); shardId.writeTo(out); - out.writeVInt(files.size()); - for (StoreFileMetaData md : files.values()) { - md.writeTo(out); - } + metadataSnapshot.writeTo(out); + } + + public String syncId() { + return metadataSnapshot.getSyncId(); } } diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index 44321fad582..4889a6ed6c4 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -22,19 +22,26 @@ package org.elasticsearch.gateway; import org.apache.lucene.util.LuceneTestCase.Slow; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.query.FilterBuilders; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; @@ -47,10 +54,7 @@ import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.*; /** * @@ -346,11 +350,13 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { @Test @Slow + @TestLogging("gateway:TRACE,indices.recovery:TRACE,index.engine:TRACE") public void testReusePeerRecovery() throws Exception { final Settings settings = settingsBuilder() .put("action.admin.cluster.node.shutdown.delay", "10ms") .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) .put("gateway.recover_after_nodes", 4) + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 4) .put(MockFSDirectoryService.CRASH_INDEX, false).build(); @@ -377,6 +383,109 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { client().admin().indices().prepareOptimize("test").setMaxNumSegments(100).get(); // just wait for merges client().admin().indices().prepareFlush().setWaitIfOngoing(true).setForce(true).get(); + boolean useSyncIds = randomBoolean(); + if (useSyncIds == false) { + logger.info("--> disabling allocation while the cluster is shut down"); + + // Disable allocations while we are closing nodes + client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(settingsBuilder() + .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) + .get(); + logger.info("--> full cluster restart"); + internalCluster().fullRestart(); + + logger.info("--> waiting for cluster to return to green after first shutdown"); + ensureGreen(); + } else { + logger.info("--> trying to sync flush"); + int numShards = Integer.parseInt(client().admin().indices().prepareGetSettings("test").get().getSetting("test", "index.number_of_shards")); + SyncedFlushService syncedFlushService = internalCluster().getInstance(SyncedFlushService.class); + for (int i = 0; i < numShards; i++) { + assertTrue(syncedFlushService.attemptSyncedFlush(new ShardId("test", i)).success()); + } + assertSyncIdsNotNull(); + } + + logger.info("--> disabling allocation while the cluster is shut down", useSyncIds ? "" : " a second time"); + // Disable allocations while we are closing nodes + client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(settingsBuilder() + .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) + .get(); + logger.info("--> full cluster restart"); + internalCluster().fullRestart(); + + logger.info("--> waiting for cluster to return to green after {}shutdown", useSyncIds ? "" : "second "); + ensureGreen(); + + if (useSyncIds) { + assertSyncIdsNotNull(); + } + RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); + for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { + RecoveryState recoveryState = response.recoveryState(); + long recovered = 0; + for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) { + if (file.name().startsWith("segments")) { + recovered += file.length(); + } + } + if (!recoveryState.getPrimary() && (useSyncIds == false)) { + logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", + response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), + recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); + assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); + assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0l)); + // we have to recover the segments file since we commit the translog ID on engine startup + assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); + assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(1)); + assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - 1)); + assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); + } else { + if (useSyncIds && !recoveryState.getPrimary()) { + logger.info("--> replica shard {} recovered from {} to {} using sync id, recovered {}, reuse {}", + response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), + recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); + } + assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0l)); + assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); + assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0)); + assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); + } + } + } + + public void assertSyncIdsNotNull() { + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + } + + @Test + @Slow + @TestLogging("indices.recovery:TRACE,index.store:TRACE") + public void testSyncFlushedRecovery() throws Exception { + final Settings settings = settingsBuilder() + .put("action.admin.cluster.node.shutdown.delay", "10ms") + .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) + .put("gateway.recover_after_nodes", 4) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 4).build(); + + internalCluster().startNodesAsync(4, settings).get(); + // prevent any rebalance actions during the recovery + assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder() + .put(indexSettings()) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE))); + ensureGreen(); + logger.info("--> indexing docs"); + for (int i = 0; i < 1000; i++) { + client().prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); + } + logger.info("--> disabling allocation while the cluster is shut down"); // Disable allocations while we are closing nodes @@ -384,52 +493,32 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { .setTransientSettings(settingsBuilder() .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) .get(); + + SyncedFlushService syncedFlushService = internalCluster().getInstance(SyncedFlushService.class); + assertTrue(syncedFlushService.attemptSyncedFlush(new ShardId("test", 0)).success()); logger.info("--> full cluster restart"); internalCluster().fullRestart(); logger.info("--> waiting for cluster to return to green after first shutdown"); ensureGreen(); - - logger.info("--> disabling allocation while the cluster is shut down second time"); - // Disable allocations while we are closing nodes - client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(settingsBuilder() - .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) - .get(); - logger.info("--> full cluster restart"); - internalCluster().fullRestart(); - - logger.info("--> waiting for cluster to return to green after second shutdown"); - ensureGreen(); - + logClusterState(); RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { RecoveryState recoveryState = response.recoveryState(); - long recovered = 0; - for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) { - if (file.name().startsWith("segments")) { - recovered += file.length(); - } - } if (!recoveryState.getPrimary()) { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); - assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); - assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0l)); - // we have to recover the segments file since we commit the translog ID on engine startup - assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes()-recovered)); - assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(1)); - assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount()-1)); - assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); } else { - assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0l)); - assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); - assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0)); - assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); + logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", + response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), + recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); } + assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(0l)); + assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); + assertThat("no files should be recovered", recoveryState.getIndex().recoveredFileCount(), equalTo(0)); + assertThat("all files should be reused", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); } - } @Test diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 1c86db6b451..3dfdb5e629c 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -2002,5 +2002,5 @@ public class InternalEngineTests extends ElasticsearchTestCase { recoveredOps.incrementAndGet(); } } - + } diff --git a/src/test/java/org/elasticsearch/index/store/StoreTest.java b/src/test/java/org/elasticsearch/index/store/StoreTest.java index c4e7ae9a7ea..2befce3fc44 100644 --- a/src/test/java/org/elasticsearch/index/store/StoreTest.java +++ b/src/test/java/org/elasticsearch/index/store/StoreTest.java @@ -32,17 +32,26 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.Version; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.io.stream.InputStreamStreamInput; +import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.Index; +import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; +import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ElasticsearchTestCase; import org.hamcrest.Matchers; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.nio.file.NoSuchFileException; @@ -51,6 +60,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.Adler32; +import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; +import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.hamcrest.Matchers.*; public class StoreTest extends ElasticsearchTestCase { @@ -180,6 +191,7 @@ public class StoreTest extends ElasticsearchTestCase { public SegmentInfo read(Directory directory, String segmentName, byte[] segmentID, IOContext context) throws IOException { return segmentInfoFormat.read(directory, segmentName, segmentID, context); } + // this sucks it's a full copy of Lucene50SegmentInfoFormat but hey I couldn't find a way to make it write 4_5_0 versions // somebody was too paranoid when implementing this. ey rmuir, was that you? - go fix it :P @Override @@ -536,7 +548,7 @@ public class StoreTest extends ElasticsearchTestCase { } final long luceneChecksum; final long adler32LegacyChecksum = adler32.getValue(); - try(IndexInput indexInput = dir.openInput("lucene_checksum.bin", IOContext.DEFAULT)) { + try (IndexInput indexInput = dir.openInput("lucene_checksum.bin", IOContext.DEFAULT)) { assertEquals(luceneFileLength, indexInput.length()); luceneChecksum = CodecUtil.retrieveChecksum(indexInput); } @@ -551,8 +563,8 @@ public class StoreTest extends ElasticsearchTestCase { } { // negative check - wrong checksum - StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength, Store.digestToString(luceneChecksum+1), Version.LUCENE_4_8_0); - StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength, Store.digestToString(adler32LegacyChecksum+1)); + StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength, Store.digestToString(luceneChecksum + 1), Version.LUCENE_4_8_0); + StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength, Store.digestToString(adler32LegacyChecksum + 1)); assertTrue(legacy.hasLegacyChecksum()); assertFalse(lucene.hasLegacyChecksum()); assertFalse(Store.checkIntegrityNoException(lucene, dir)); @@ -560,8 +572,8 @@ public class StoreTest extends ElasticsearchTestCase { } { // negative check - wrong length - StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength+1, Store.digestToString(luceneChecksum), Version.LUCENE_4_8_0); - StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength+1, Store.digestToString(adler32LegacyChecksum)); + StoreFileMetaData lucene = new StoreFileMetaData("lucene_checksum.bin", luceneFileLength + 1, Store.digestToString(luceneChecksum), Version.LUCENE_4_8_0); + StoreFileMetaData legacy = new StoreFileMetaData("legacy.bin", legacyFileLength + 1, Store.digestToString(adler32LegacyChecksum)); assertTrue(legacy.hasLegacyChecksum()); assertFalse(lucene.hasLegacyChecksum()); assertFalse(Store.checkIntegrityNoException(lucene, dir)); @@ -616,19 +628,19 @@ public class StoreTest extends ElasticsearchTestCase { IOUtils.close(dir); } - private void readIndexInputFullyWithRandomSeeks(IndexInput indexInput) throws IOException{ + private void readIndexInputFullyWithRandomSeeks(IndexInput indexInput) throws IOException { BytesRef ref = new BytesRef(scaledRandomIntBetween(1, 1024)); long pos = 0; while (pos < indexInput.length()) { assertEquals(pos, indexInput.getFilePointer()); int op = random().nextInt(5); - if (op == 0 ) { - int shift = 100 - randomIntBetween(0, 200); - pos = Math.min(indexInput.length() - 1, Math.max(0, pos + shift)); + if (op == 0) { + int shift = 100 - randomIntBetween(0, 200); + pos = Math.min(indexInput.length() - 1, Math.max(0, pos + shift)); indexInput.seek(pos); } else if (op == 1) { indexInput.readByte(); - pos ++; + pos++; } else { int min = (int) Math.min(indexInput.length() - pos, ref.bytes.length); indexInput.readBytes(ref.bytes, ref.offset, min); @@ -673,16 +685,18 @@ public class StoreTest extends ElasticsearchTestCase { public LuceneManagedDirectoryService(Random random) { this(random, true); } + public LuceneManagedDirectoryService(Random random, boolean preventDoubleWrite) { super(new ShardId("fake", 1), ImmutableSettings.EMPTY); - dir = StoreTest.newDirectory(random); - if (dir instanceof MockDirectoryWrapper) { - ((MockDirectoryWrapper)dir).setPreventDoubleWrite(preventDoubleWrite); - // TODO: fix this test to handle virus checker - ((MockDirectoryWrapper)dir).setEnableVirusScanner(false); - } + dir = StoreTest.newDirectory(random); + if (dir instanceof MockDirectoryWrapper) { + ((MockDirectoryWrapper) dir).setPreventDoubleWrite(preventDoubleWrite); + // TODO: fix this test to handle virus checker + ((MockDirectoryWrapper) dir).setEnableVirusScanner(false); + } this.random = random; } + @Override public Directory newDirectory() throws IOException { return dir; @@ -711,11 +725,11 @@ public class StoreTest extends ElasticsearchTestCase { @Test public void testRecoveryDiffWithLegacyCommit() { Map metaDataMap = new HashMap<>(); - metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[] {1}))); + metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); - Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); Store.RecoveryDiff recoveryDiff = first.recoveryDiff(second); assertEquals(recoveryDiff.toString(), recoveryDiff.different.size(), 2); } @@ -760,7 +774,7 @@ public class StoreTest extends ElasticsearchTestCase { store.close(); } long time = new Date().getTime(); - while(time == new Date().getTime()) { + while (time == new Date().getTime()) { Thread.sleep(10); // bump the time } Store.MetadataSnapshot second; @@ -827,7 +841,7 @@ public class StoreTest extends ElasticsearchTestCase { } Store.RecoveryDiff afterDeleteDiff = metadata.recoveryDiff(second); if (delFile != null) { - assertThat(afterDeleteDiff.identical.size(), equalTo(metadata.size()-2)); // segments_N + del file + assertThat(afterDeleteDiff.identical.size(), equalTo(metadata.size() - 2)); // segments_N + del file assertThat(afterDeleteDiff.different.size(), equalTo(0)); assertThat(afterDeleteDiff.missing.size(), equalTo(2)); } else { @@ -856,7 +870,7 @@ public class StoreTest extends ElasticsearchTestCase { Store.MetadataSnapshot newCommitMetaData = store.getMetadata(); Store.RecoveryDiff newCommitDiff = newCommitMetaData.recoveryDiff(metadata); if (delFile != null) { - assertThat(newCommitDiff.identical.size(), equalTo(newCommitMetaData.size()-5)); // segments_N, del file, cfs, cfe, si for the new segment + assertThat(newCommitDiff.identical.size(), equalTo(newCommitMetaData.size() - 5)); // segments_N, del file, cfs, cfe, si for the new segment assertThat(newCommitDiff.different.size(), equalTo(1)); // the del file must be different assertThat(newCommitDiff.different.get(0).name(), endsWith(".liv")); assertThat(newCommitDiff.missing.size(), equalTo(4)); // segments_N,cfs, cfe, si for the new segment @@ -883,7 +897,7 @@ public class StoreTest extends ElasticsearchTestCase { int docs = 1 + random().nextInt(100); int numCommits = 0; for (int i = 0; i < docs; i++) { - if (i > 0 && randomIntBetween(0, 10 ) == 0) { + if (i > 0 && randomIntBetween(0, 10) == 0) { writer.commit(); numCommits++; } @@ -948,7 +962,7 @@ public class StoreTest extends ElasticsearchTestCase { assertTrue(firstMeta.contains(file) || Store.isChecksum(file) || file.equals("write.lock")); if (Store.isChecksum(file)) { numChecksums++; - } else if (secondMeta.contains(file) == false) { + } else if (secondMeta.contains(file) == false) { numNotFound++; } @@ -967,7 +981,7 @@ public class StoreTest extends ElasticsearchTestCase { assertTrue(file, secondMeta.contains(file) || Store.isChecksum(file) || file.equals("write.lock")); if (Store.isChecksum(file)) { numChecksums++; - } else if (firstMeta.contains(file) == false) { + } else if (firstMeta.contains(file) == false) { numNotFound++; } @@ -985,7 +999,7 @@ public class StoreTest extends ElasticsearchTestCase { Map metaDataMap = new HashMap<>(); metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap); + Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); final ShardId shardId = new ShardId(new Index("index"), 1); DirectoryService directoryService = new LuceneManagedDirectoryService(random()); @@ -1009,7 +1023,7 @@ public class StoreTest extends ElasticsearchTestCase { final AtomicInteger count = new AtomicInteger(0); final ShardLock lock = new DummyShardLock(shardId); - Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, lock , new Store.OnClose() { + Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, lock, new Store.OnClose() { @Override public void handle(ShardLock theLock) { assertEquals(shardId, theLock.getShardId()); @@ -1081,4 +1095,95 @@ public class StoreTest extends ElasticsearchTestCase { } return numNonExtra; } + + @Test + public void testMetadataSnapshotStreaming() throws Exception { + + Store.MetadataSnapshot outMetadataSnapshot = createMetaDataSnapshot(); + org.elasticsearch.Version targetNodeVersion = randomVersion(random()); + + ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); + out.setVersion(targetNodeVersion); + outMetadataSnapshot.writeTo(out); + + ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); + InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); + in.setVersion(targetNodeVersion); + Store.MetadataSnapshot inMetadataSnapshot = new Store.MetadataSnapshot(in); + Map origEntries = new HashMap<>(); + origEntries.putAll(outMetadataSnapshot.asMap()); + for (Map.Entry entry : inMetadataSnapshot.asMap().entrySet()) { + assertThat(entry.getValue().name(), equalTo(origEntries.remove(entry.getKey()).name())); + } + assertThat(origEntries.size(), equalTo(0)); + assertThat(inMetadataSnapshot.getCommitUserData(), equalTo(outMetadataSnapshot.getCommitUserData())); + } + + protected Store.MetadataSnapshot createMetaDataSnapshot() { + StoreFileMetaData storeFileMetaData1 = new StoreFileMetaData("segments", 1); + StoreFileMetaData storeFileMetaData2 = new StoreFileMetaData("no_segments", 1); + Map storeFileMetaDataMap = new HashMap<>(); + storeFileMetaDataMap.put(storeFileMetaData1.name(), storeFileMetaData1); + storeFileMetaDataMap.put(storeFileMetaData2.name(), storeFileMetaData2); + Map commitUserData = new HashMap<>(); + commitUserData.put("userdata_1", "test"); + commitUserData.put("userdata_2", "test"); + return new Store.MetadataSnapshot(storeFileMetaDataMap, commitUserData); + } + + @Test + public void testUserDataRead() throws IOException { + final ShardId shardId = new ShardId(new Index("index"), 1); + DirectoryService directoryService = new LuceneManagedDirectoryService(random()); + Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId)); + IndexWriterConfig config = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()); + SnapshotDeletionPolicy deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastDeletionPolicy(shardId, EMPTY_SETTINGS)); + config.setIndexDeletionPolicy(deletionPolicy); + IndexWriter writer = new IndexWriter(store.directory(), config); + Document doc = new Document(); + doc.add(new TextField("id", "1", Field.Store.NO)); + writer.addDocument(doc); + Map commitData = new HashMap<>(2); + String syncId = "a sync id"; + String translogId = "a translog id"; + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + commitData.put(Translog.TRANSLOG_ID_KEY, translogId); + writer.setCommitData(commitData); + writer.commit(); + writer.close(); + Store.MetadataSnapshot metadata; + if (randomBoolean()) { + metadata = store.getMetadata(); + } else { + metadata = store.getMetadata(deletionPolicy.snapshot()); + } + assertFalse(metadata.asMap().isEmpty()); + // do not check for correct files, we have enough tests for that above + assertThat(metadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); + assertThat(metadata.getCommitUserData().get(Translog.TRANSLOG_ID_KEY), equalTo(translogId)); + TestUtil.checkIndex(store.directory()); + assertDeleteContent(store, directoryService); + IOUtils.close(store); + } + + @Test + public void testStreamStoreFilesMetaData() throws Exception { + Store.MetadataSnapshot metadataSnapshot = createMetaDataSnapshot(); + TransportNodesListShardStoreMetaData.StoreFilesMetaData outStoreFileMetaData = new TransportNodesListShardStoreMetaData.StoreFilesMetaData(randomBoolean(), new ShardId("test", 0),metadataSnapshot); + ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); + org.elasticsearch.Version targetNodeVersion = randomVersion(random()); + out.setVersion(targetNodeVersion); + outStoreFileMetaData.writeTo(out); + ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); + InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); + in.setVersion(targetNodeVersion); + TransportNodesListShardStoreMetaData.StoreFilesMetaData inStoreFileMetaData = TransportNodesListShardStoreMetaData.StoreFilesMetaData.readStoreFilesMetaData(in); + Iterator outFiles = outStoreFileMetaData.iterator(); + for (StoreFileMetaData inFile : inStoreFileMetaData) { + assertThat(inFile.name(), equalTo(outFiles.next().name())); + } + assertThat(outStoreFileMetaData.syncId(), equalTo(inStoreFileMetaData.syncId())); + } } diff --git a/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java b/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java index c15d1d8b552..4a1586e5c45 100644 --- a/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java +++ b/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTest.java @@ -19,12 +19,14 @@ package org.elasticsearch.indices.recovery; +import org.apache.lucene.index.IndexFileNames; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; @@ -32,6 +34,8 @@ import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.hamcrest.Matchers.equalTo; @@ -49,7 +53,7 @@ public class StartRecoveryRequestTest extends ElasticsearchTestCase { new DiscoveryNode("a", new LocalTransportAddress("1"), targetNodeVersion), new DiscoveryNode("b", new LocalTransportAddress("1"), targetNodeVersion), true, - Collections.emptyMap(), + Store.MetadataSnapshot.EMPTY, RecoveryState.Type.RELOCATION, 1l @@ -69,11 +73,9 @@ public class StartRecoveryRequestTest extends ElasticsearchTestCase { assertThat(outRequest.sourceNode(), equalTo(inRequest.sourceNode())); assertThat(outRequest.targetNode(), equalTo(inRequest.targetNode())); assertThat(outRequest.markAsRelocated(), equalTo(inRequest.markAsRelocated())); - assertThat(outRequest.existingFiles(), equalTo(inRequest.existingFiles())); + assertThat(outRequest.metadataSnapshot().asMap(), equalTo(inRequest.metadataSnapshot().asMap())); assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); assertThat(outRequest.recoveryType(), equalTo(inRequest.recoveryType())); } - - } From fbc26ca49b5059114762606e0e6429eddfdd9c01 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 5 May 2015 16:15:25 +0200 Subject: [PATCH 07/38] implement check for in flight now that we have the counter --- .../java/org/elasticsearch/indices/SyncedFlushService.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 4ec05620863..0594797bba4 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -104,7 +104,7 @@ public class SyncedFlushService extends AbstractComponent { } int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); - if (inflight < 0 || inflight > 0) { + if (inflight != 1) { return new SyncedFlushResult("operation counter on primary is non zero [" + inflight + "]"); } @@ -307,8 +307,7 @@ public class SyncedFlushService extends AbstractComponent { if (indexShard.routingEntry().primary() == false) { throw new IndexShardException(request.shardId(), "expected a primary shard"); } - // nocommit - implement :) - int opCount = 0; + int opCount = indexShard.getOperationsCount(); logger.trace("{} in flight operations sampled at [{}]", request.shardId(), opCount); return new InFlightOpsResponse(opCount); } From 8349a3ee1e83a12e3aa8e4d6820216fc04b8d13b Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 5 May 2015 17:19:09 +0200 Subject: [PATCH 08/38] call sync flush when shard is marked as inactive --- .../elasticsearch/index/shard/IndexShard.java | 1 + .../elasticsearch/indices/IndicesLifecycle.java | 9 +++++++++ .../indices/InternalIndicesLifecycle.java | 11 +++++++++++ .../indices/SyncedFlushService.java | 11 ++++++++++- .../index/shard/IndexShardTests.java | 16 +++++++++++++++- .../IndicesLifecycleListenerSingleNodeTests.java | 2 +- 6 files changed, 47 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fad982e1437..8a084927c62 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -999,6 +999,7 @@ public class IndexShard extends AbstractIndexShardComponent { public void markAsInactive() { updateBufferSize(EngineConfig.INACTIVE_SHARD_INDEXING_BUFFER, Translog.INACTIVE_SHARD_TRANSLOG_BUFFER); + indicesLifecycle.onShardInactive(this); } public final boolean isFlushOnClose() { diff --git a/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java b/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java index 003ba9f9264..39de271bfe8 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java +++ b/src/main/java/org/elasticsearch/indices/IndicesLifecycle.java @@ -201,6 +201,15 @@ public interface IndicesLifecycle { public void beforeIndexDeleted(IndexService indexService) { } + + /** + * Called when a shard is marked as inactive + * + * @param indexShard The shard that was marked inactive + */ + public void onShardInactive(IndexShard indexShard) { + + } } } diff --git a/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java b/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java index 8e7e6527bef..046acb7f625 100644 --- a/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java +++ b/src/main/java/org/elasticsearch/indices/InternalIndicesLifecycle.java @@ -245,4 +245,15 @@ public class InternalIndicesLifecycle extends AbstractComponent implements Indic } } } + + public void onShardInactive(IndexShard indexShard) { + for (Listener listener : listeners) { + try { + listener.onShardInactive(indexShard); + } catch (Throwable t) { + logger.warn("{} failed to invoke on shard inactive callback", t, indexShard.shardId()); + throw t; + } + } + } } diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 0594797bba4..ad4c47921a0 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -84,9 +84,18 @@ public class SyncedFlushService extends AbstractComponent { preSyncTimeout = settings.getAsTime(SETTING_PRE_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); syncTimeout = settings.getAsTime(SETTING_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); inflightOpsTimeout = settings.getAsTime(SETTING_IN_FLIGHT_OPS_TIMEOUT, TimeValue.timeValueMinutes(5)); + indicesService.indicesLifecycle().addListener(new IndicesLifecycle.Listener() { + @Override + public void onShardInactive(IndexShard indexShard) { + // we only want to call sync flush once, so only trigger it when we are on a primary + if (indexShard.routingEntry().primary()) { + attemptSyncedFlush(indexShard.shardId()); + } + } + }); } - public SyncedFlushResult attemptSyncedFlush(ShardId shardId) throws ExecutionException, InterruptedException { + public SyncedFlushResult attemptSyncedFlush(ShardId shardId) { final ClusterState state = clusterService.state(); final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); if (indexRoutingTable == null) { diff --git a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 8464cf8e42a..544a3472f96 100644 --- a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.shard; +import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.MutableShardRouting; import org.elasticsearch.cluster.routing.ShardRouting; @@ -26,6 +27,7 @@ import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.ElasticsearchSingleNodeTest; import org.junit.Test; @@ -37,6 +39,8 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.ExecutionException; +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.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; @@ -225,8 +229,18 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest { indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); assertEquals(1, indexShard.getOperationsCount()); + } - + @Test + public void testMarkAsInactiveTriggersSyncedFlush() { + assertAcked(client().admin().indices().prepareCreate("test") + .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0)); + client().prepareIndex("test", "test").setSource("{}").get(); + ensureGreen("test"); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + indicesService.indexService("test").shard(0).markAsInactive(); + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); } public static ShardStateMetaData load(ESLogger logger, Path... shardPaths) throws IOException { diff --git a/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index afa9e0e6cf4..910902b7b4b 100644 --- a/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -86,5 +86,5 @@ public class IndicesLifecycleListenerSingleNodeTests extends ElasticsearchSingle assertAcked(client().admin().indices().prepareDelete("test").get()); assertEquals(7, counter.get()); } - + } From e3d346d3b72cd02c9585889d8eaf89cb2e0d12e1 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 5 May 2015 17:32:03 +0200 Subject: [PATCH 09/38] cleanup --- .../recovery/RecoverySourceHandler.java | 12 ---- .../gateway/RecoveryFromGatewayTests.java | 58 ------------------- 2 files changed, 70 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index f7af51628b1..8eb73e41c9e 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -522,19 +522,7 @@ public class RecoverySourceHandler { } /** -<<<<<<< HEAD - * Perform phase 3 of the recovery process - *

    - * Phase3 again takes a snapshot of the translog, however this time the - * snapshot is acquired under a write lock. The translog operations are - * sent to the target node where they are replayed. - *

    - * {@code InternalEngine#recover} is responsible for taking the snapshot - * of the translog, and after phase 3 completes the snapshots from all - * three phases are released. -======= * finalizes the recovery process ->>>>>>> origin/master */ public void finalizeRecovery() { if (shard.state() == IndexShardState.CLOSED) { diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index 4889a6ed6c4..9d3c1943887 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -463,64 +463,6 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { } } - @Test - @Slow - @TestLogging("indices.recovery:TRACE,index.store:TRACE") - public void testSyncFlushedRecovery() throws Exception { - final Settings settings = settingsBuilder() - .put("action.admin.cluster.node.shutdown.delay", "10ms") - .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) - .put("gateway.recover_after_nodes", 4) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 4).build(); - - internalCluster().startNodesAsync(4, settings).get(); - // prevent any rebalance actions during the recovery - assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder() - .put(indexSettings()) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE))); - ensureGreen(); - logger.info("--> indexing docs"); - for (int i = 0; i < 1000; i++) { - client().prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); - } - - logger.info("--> disabling allocation while the cluster is shut down"); - - // Disable allocations while we are closing nodes - client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(settingsBuilder() - .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) - .get(); - - SyncedFlushService syncedFlushService = internalCluster().getInstance(SyncedFlushService.class); - assertTrue(syncedFlushService.attemptSyncedFlush(new ShardId("test", 0)).success()); - logger.info("--> full cluster restart"); - internalCluster().fullRestart(); - - logger.info("--> waiting for cluster to return to green after first shutdown"); - ensureGreen(); - logClusterState(); - RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); - for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { - RecoveryState recoveryState = response.recoveryState(); - if (!recoveryState.getPrimary()) { - logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", - response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), - recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); - } else { - logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", - response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), - recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); - } - assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(0l)); - assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); - assertThat("no files should be recovered", recoveryState.getIndex().recoveredFileCount(), equalTo(0)); - assertThat("all files should be reused", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); - } - } - @Test @Slow public void testRecoveryDifferentNodeOrderStartup() throws Exception { From dc2eed9681d6d5a3eec43f4b3d777e73effe8efd Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 5 May 2015 17:45:41 +0200 Subject: [PATCH 10/38] fix after merge with master --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 8eb73e41c9e..93b6e5f030b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -417,7 +417,7 @@ public class RecoverySourceHandler { try { final Store.MetadataSnapshot remainingFilesAfterCleanup = recoverWithSyncId? request.metadataSnapshot(): recoverySourceMetadata; transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, - new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, translogView.totalOperations()), + new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), remainingFilesAfterCleanup, translogView.totalOperations()), TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); } catch (RemoteTransportException remoteException) { From a7abe0a324997c1c884c1278306cdba05be9594c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 12 May 2015 10:03:12 +0200 Subject: [PATCH 11/38] Make synced flush attemp async to run it easily from a TransportAction Today we enforce blocking which doesnt' really fit in the elasticsearch model this commit adds async execution to the synced flush service by passing a ActinListener to the service returing immediately. --- .../indices/SyncedFlushService.java | 113 +++++++++++------- .../gateway/RecoveryFromGatewayTests.java | 3 +- .../index/shard/IndexShardTests.java | 9 +- .../org/elasticsearch/indices/FlushTest.java | 3 +- .../elasticsearch/test/SyncedFlushUtil.java | 67 +++++++++++ 5 files changed, 149 insertions(+), 46 deletions(-) create mode 100644 src/test/java/org/elasticsearch/test/SyncedFlushUtil.java diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index ad4c47921a0..a15ad796da8 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -18,9 +18,14 @@ */ package org.elasticsearch.indices; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.admin.cluster.node.liveness.LivenessRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.action.support.DelegatingActionListener; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -34,7 +39,9 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexShardMissingException; import org.elasticsearch.index.engine.Engine; @@ -52,6 +59,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; public class SyncedFlushService extends AbstractComponent { @@ -86,41 +94,53 @@ public class SyncedFlushService extends AbstractComponent { inflightOpsTimeout = settings.getAsTime(SETTING_IN_FLIGHT_OPS_TIMEOUT, TimeValue.timeValueMinutes(5)); indicesService.indicesLifecycle().addListener(new IndicesLifecycle.Listener() { @Override - public void onShardInactive(IndexShard indexShard) { + public void onShardInactive(final IndexShard indexShard) { // we only want to call sync flush once, so only trigger it when we are on a primary if (indexShard.routingEntry().primary()) { - attemptSyncedFlush(indexShard.shardId()); + attemptSyncedFlush(indexShard.shardId(), new ActionListener() { + @Override + public void onResponse(SyncedFlushResult syncedFlushResult) { + logger.debug("{} sync flush on inactive shard returned successfully for sync_id: {}", syncedFlushResult.getShardId(), syncedFlushResult.syncId()); + } + + @Override + public void onFailure(Throwable e) { + logger.debug("{} sync flush on inactive shard failed", e, indexShard.shardId()); + } + }); } } }); } - public SyncedFlushResult attemptSyncedFlush(ShardId shardId) { - final ClusterState state = clusterService.state(); - final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); - if (indexRoutingTable == null) { - throw new IndexMissingException(shardId.index()); - } - final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); - if (shardRoutingTable == null) { - throw new IndexShardMissingException(shardId); - } - final List activeShards = shardRoutingTable.activeShards(); - Map commitIds = sendPreSyncRequests(activeShards, state, shardId); + public void attemptSyncedFlush(ShardId shardId, ActionListener actionListener) { + try { + final ClusterState state = clusterService.state(); + final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); + if (indexRoutingTable == null) { + throw new IndexMissingException(shardId.index()); + } + final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); + if (shardRoutingTable == null) { + throw new IndexShardMissingException(shardId); + } + final List activeShards = shardRoutingTable.activeShards(); + Map commitIds = sendPreSyncRequests(activeShards, state, shardId); - if (commitIds.isEmpty()) { - return new SyncedFlushResult("all shards failed to commit on pre-sync"); + if (commitIds.isEmpty()) { + actionListener.onResponse(new SyncedFlushResult(shardId, "all shards failed to commit on pre-sync")); + } + + int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); + if (inflight != 1) { + actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); + } + + String syncId = Strings.base64UUID(); + sendSyncRequests(syncId, activeShards, state, commitIds, shardId, actionListener); + } catch (Throwable t) { + actionListener.onFailure(t); } - - int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); - if (inflight != 1) { - return new SyncedFlushResult("operation counter on primary is non zero [" + inflight + "]"); - } - - String syncId = Strings.base64UUID(); - Map results = sendSyncRequests(syncId, activeShards, state, commitIds, shardId); - - return new SyncedFlushResult(syncId, results); } /** @@ -167,29 +187,32 @@ public class SyncedFlushService extends AbstractComponent { } catch (InterruptedException e) { logger.debug("{} interrupted while waiting for in flight operation check", shardId); } - final int count = result.get(); logger.trace("{} in flight operation count [{}]", shardId, count); return count; } - private Map sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId) { - final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); + private void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { + final CountDown countDownLatch = new CountDown(shards.size()); final Map results = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); if (node == null) { logger.trace("{} is assigned to an unknown node. skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); results.put(shard, new SyncedFlushResponse("unknown node")); - countDownLatch.countDown(); + if (countDownLatch.countDown()) { + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } continue; } final byte[] expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); if (expectedCommitId == null) { logger.trace("{} can't resolve expected commit id for {}, skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); results.put(shard, new SyncedFlushResponse("no commit id from pre-sync flush")); - countDownLatch.countDown(); + if (countDownLatch.countDown()) { + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } continue; } logger.trace("{} sending synced flush request to {}. sync id [{}].", shardId, shard, syncId); @@ -205,14 +228,18 @@ public class SyncedFlushService extends AbstractComponent { SyncedFlushResponse existing = results.put(shard, response); assert existing == null : "got two answers for node [" + node + "]"; // count after the assert so we won't decrement twice in handleException - countDownLatch.countDown(); + if (countDownLatch.countDown()) { + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } } @Override public void handleException(TransportException exp) { logger.trace("{} error while performing synced flush on [{}], skipping", exp, shardId, shard); results.put(shard, new SyncedFlushResponse(exp.getMessage())); - countDownLatch.countDown(); + if (countDownLatch.countDown()) { + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } } @Override @@ -221,15 +248,7 @@ public class SyncedFlushService extends AbstractComponent { } }); } - try { - if (countDownLatch.await(syncTimeout.millis(), TimeUnit.MILLISECONDS) == false) { - logger.debug("{} waiting for synced flush with id [{}] timed out after [{}]. pending ops [{}]", shardId, syncId, syncTimeout, countDownLatch.getCount()); - } - } catch (InterruptedException e) { - logger.debug("{} interrupted while waiting for sync requests (sync id [{}])", shardId, syncId); - } - return results; } /** @@ -326,22 +345,30 @@ public class SyncedFlushService extends AbstractComponent { private final Map shardResponses; private final String syncId; + public ShardId getShardId() { + return shardId; + } + + private final ShardId shardId; + /** * failure constructor */ - SyncedFlushResult(String failureReason) { + SyncedFlushResult(ShardId shardId, String failureReason) { this.syncId = null; this.failureReason = failureReason; this.shardResponses = new HashMap<>(); + this.shardId = shardId; } /** * success constructor */ - SyncedFlushResult(String syncId, Map shardResponses) { + SyncedFlushResult(ShardId shardId, String syncId, Map shardResponses) { this.failureReason = null; this.shardResponses = shardResponses; this.syncId = syncId; + this.shardId = shardId; } public boolean success() { diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index 943ed50415d..6ee3448b99f 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.test.SyncedFlushUtil; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; @@ -401,7 +402,7 @@ public class RecoveryFromGatewayTests extends ElasticsearchIntegrationTest { int numShards = Integer.parseInt(client().admin().indices().prepareGetSettings("test").get().getSetting("test", "index.number_of_shards")); SyncedFlushService syncedFlushService = internalCluster().getInstance(SyncedFlushService.class); for (int i = 0; i < numShards; i++) { - assertTrue(syncedFlushService.attemptSyncedFlush(new ShardId("test", i)).success()); + assertTrue(SyncedFlushUtil.attemptSyncedFlush(syncedFlushService, new ShardId("test", i)).success()); } assertSyncIdsNotNull(); } diff --git a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b783f731bec..b1b3aa1ee1a 100644 --- a/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -233,13 +233,20 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest { } @Test - public void testMarkAsInactiveTriggersSyncedFlush() { + public void testMarkAsInactiveTriggersSyncedFlush() throws Exception { assertAcked(client().admin().indices().prepareCreate("test") .setSettings(SETTING_NUMBER_OF_SHARDS, 1, SETTING_NUMBER_OF_REPLICAS, 0)); client().prepareIndex("test", "test").setSource("{}").get(); ensureGreen("test"); IndicesService indicesService = getInstanceFromNode(IndicesService.class); indicesService.indexService("test").shard(0).markAsInactive(); + assertBusy(new Runnable() { // should be very very quick + @Override + public void run() { + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + }); IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); assertNotNull(indexStats.getShards()[0].getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); } diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index a4a9b4d2171..500579aeb3c 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -26,6 +26,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.SyncedFlushUtil; import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; @@ -88,7 +89,7 @@ public class FlushTest extends ElasticsearchIntegrationTest { assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); } - SyncedFlushService.SyncedFlushResult result = internalCluster().getInstance(SyncedFlushService.class).attemptSyncedFlush(new ShardId("test", 0)); + SyncedFlushService.SyncedFlushResult result = SyncedFlushUtil.attemptSyncedFlush(internalCluster().getInstance(SyncedFlushService.class), new ShardId("test", 0)); assertTrue(result.success()); assertThat(result.totalShards(), equalTo(indexStats.getShards().length)); assertThat(result.successfulShards(), equalTo(indexStats.getShards().length)); diff --git a/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java b/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java new file mode 100644 index 00000000000..53f1941cfff --- /dev/null +++ b/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java @@ -0,0 +1,67 @@ +/* + * 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.test; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +/** Utils for SyncedFlush */ +public class SyncedFlushUtil { + + private SyncedFlushUtil() { + + } + + /** + * Blocking version of {@link SyncedFlushService#attemptSyncedFlush(ShardId, ActionListener)} + */ + public static SyncedFlushService.SyncedFlushResult attemptSyncedFlush(SyncedFlushService service, ShardId shardId) { + final CountDownLatch countDown = new CountDownLatch(1); + final AtomicReference result = new AtomicReference<>(); + final AtomicReference exception = new AtomicReference<>(); + service.attemptSyncedFlush(shardId, new ActionListener() { + @Override + public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { + result.compareAndSet(null, syncedFlushResult); + countDown.countDown(); + } + + @Override + public void onFailure(Throwable e) { + exception.compareAndSet(null, e); + countDown.countDown(); + } + }); + try { + countDown.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (exception.get() != null) { + throw ExceptionsHelper.convertToElastic(exception.get()); + } + return result.get(); + } + +} From c628d67f9e0389ea1c02c68e8fdb29c70d1596f6 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 5 May 2015 18:12:40 +0200 Subject: [PATCH 12/38] api for synced flush closes #11098 --- rest-api-spec/api/indices.seal.json | 17 +++ rest-api-spec/test/indices.seal/10_basic.yaml | 16 +++ .../elasticsearch/action/ActionModule.java | 3 + .../admin/indices/seal/SealIndicesAction.java | 45 +++++++ .../indices/seal/SealIndicesRequest.java | 99 ++++++++++++++ .../seal/SealIndicesRequestBuilder.java | 38 ++++++ .../indices/seal/SealIndicesResponse.java | 122 ++++++++++++++++++ .../seal/TransportSealIndicesAction.java | 88 +++++++++++++ .../client/IndicesAdminClient.java | 25 ++++ .../client/support/AbstractClient.java | 19 +++ .../indices/SyncedFlushService.java | 66 +++++++--- .../rest/action/RestActionModule.java | 6 +- .../indices/seal/RestSealIndicesAction.java | 67 ++++++++++ .../admin/indices/seal/SealIndicesTests.java | 117 +++++++++++++++++ .../org/elasticsearch/indices/FlushTest.java | 122 +++++++++++++++++- .../test/ElasticsearchIntegrationTest.java | 10 +- 16 files changed, 837 insertions(+), 23 deletions(-) create mode 100644 rest-api-spec/api/indices.seal.json create mode 100644 rest-api-spec/test/indices.seal/10_basic.yaml create mode 100644 src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java create mode 100644 src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java create mode 100644 src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java create mode 100644 src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java create mode 100644 src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java create mode 100644 src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java create mode 100644 src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java diff --git a/rest-api-spec/api/indices.seal.json b/rest-api-spec/api/indices.seal.json new file mode 100644 index 00000000000..3377d6e88a1 --- /dev/null +++ b/rest-api-spec/api/indices.seal.json @@ -0,0 +1,17 @@ +{ + "indices.seal": { + "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-seal.html", + "methods": ["POST", "GET"], + "url": { + "path": "/_seal", + "paths": ["/_seal", "/{index}/_seal"], + "parts": { + "index": { + "type" : "list", + "description" : "A comma-separated list of index names; use `_all` or empty string for all indices" + } + } + }, + "body": null + } +} diff --git a/rest-api-spec/test/indices.seal/10_basic.yaml b/rest-api-spec/test/indices.seal/10_basic.yaml new file mode 100644 index 00000000000..2291cf3d647 --- /dev/null +++ b/rest-api-spec/test/indices.seal/10_basic.yaml @@ -0,0 +1,16 @@ +--- +"Index seal rest test": +- do: + indices.create: + index: testing + +- do: + cluster.health: + wait_for_status: green +- do: + indices.seal: + index: testing +- do: + indices.stats: {level: shards} + +- is_true: indices.testing.shards.0.0.commit.user_data.sync_id diff --git a/src/main/java/org/elasticsearch/action/ActionModule.java b/src/main/java/org/elasticsearch/action/ActionModule.java index c529a3e876f..1c273f67b5b 100644 --- a/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/src/main/java/org/elasticsearch/action/ActionModule.java @@ -103,6 +103,8 @@ import org.elasticsearch.action.admin.indices.settings.put.TransportUpdateSettin import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.TransportSealIndicesAction; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesAction; @@ -254,6 +256,7 @@ public class ActionModule extends AbstractModule { registerAction(ValidateQueryAction.INSTANCE, TransportValidateQueryAction.class); registerAction(RefreshAction.INSTANCE, TransportRefreshAction.class); registerAction(FlushAction.INSTANCE, TransportFlushAction.class); + registerAction(SealIndicesAction.INSTANCE, TransportSealIndicesAction.class); registerAction(OptimizeAction.INSTANCE, TransportOptimizeAction.class); registerAction(ClearIndicesCacheAction.INSTANCE, TransportClearIndicesCacheAction.class); registerAction(PutWarmerAction.INSTANCE, TransportPutWarmerAction.class); diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java new file mode 100644 index 00000000000..e2f230defaf --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java @@ -0,0 +1,45 @@ +/* + * 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.seal; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.ElasticsearchClient; + +/** + */ +public class SealIndicesAction extends Action { + + public static final SealIndicesAction INSTANCE = new SealIndicesAction(); + public static final String NAME = "indices:admin/sealindices"; + + private SealIndicesAction() { + super(NAME); + } + + @Override + public SealIndicesResponse newResponse() { + return new SealIndicesResponse(); + } + + @Override + public SealIndicesRequestBuilder newRequestBuilder(ElasticsearchClient client) { + return new SealIndicesRequestBuilder(client, this); + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java new file mode 100644 index 00000000000..e8f032ea7ae --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java @@ -0,0 +1,99 @@ +/* + * 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.seal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Arrays; + +/** + * A request to seal one or more indices. + */ +public class SealIndicesRequest extends ActionRequest implements IndicesRequest.Replaceable { + + private String[] indices; + + private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed(); + + SealIndicesRequest() { + } + + /** + * Constructs a seal request against one or more indices. If nothing is provided, all indices will + * be sealed. + */ + public SealIndicesRequest(String... indices) { + this.indices = indices; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArrayNullable(indices); + indicesOptions.writeIndicesOptions(out); + } + + @Override + public String toString() { + return "SealIndicesRequest{" + + "indices=" + Arrays.toString(indices) + + ", indicesOptions=" + indicesOptions + + '}'; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + indices = in.readStringArray(); + indicesOptions = IndicesOptions.readIndicesOptions(in); + } + + @Override + public SealIndicesRequest indices(String[] indices) { + this.indices = indices; + return this; + } + + public String[] indices() { + return indices; + } + + @Override + public IndicesOptions indicesOptions() { + return indicesOptions; + } + + @SuppressWarnings("unchecked") + public final SealIndicesRequest indicesOptions(IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + return this; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java new file mode 100644 index 00000000000..a424ab3fc3b --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequestBuilder.java @@ -0,0 +1,38 @@ +/* + * 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.seal; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.ElasticsearchClient; + +/** + * + */ +public class SealIndicesRequestBuilder extends ActionRequestBuilder { + + public SealIndicesRequestBuilder(ElasticsearchClient client, SealIndicesAction action) { + super(client, action, new SealIndicesRequest()); + } + + public SealIndicesRequestBuilder indices(String ... indices) { + request.indices(indices); + return this; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java new file mode 100644 index 00000000000..c4e895b5cc8 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java @@ -0,0 +1,122 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.admin.indices.seal; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.indices.SyncedFlushService; + +import java.io.IOException; +import java.util.*; + +/** + * A response to a seal action on several indices. + */ +public class SealIndicesResponse extends ActionResponse implements ToXContent { + + private Set results; + + SealIndicesResponse() { + } + + SealIndicesResponse(Set results) { + this.results = results; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + int size = in.readVInt(); + results = new HashSet<>(); + for (int i = 0; i < size; i++) { + SyncedFlushService.SyncedFlushResult syncedFlushResult = new SyncedFlushService.SyncedFlushResult(); + syncedFlushResult.readFrom(in); + results.add(syncedFlushResult); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(results.size()); + for (SyncedFlushService.SyncedFlushResult syncedFlushResult : results) { + syncedFlushResult.writeTo(out); + } + } + + public Set results() { + return results; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + Map> allResults = new HashMap<>(); + + // first, sort everything by index and shard id + for (SyncedFlushService.SyncedFlushResult result : results) { + String indexName = result.getShardId().index().name(); + int shardId = result.getShardId().getId(); + + if (allResults.get(indexName) == null) { + // no results yet for this index + allResults.put(indexName, new TreeMap()); + } + if (result.shardResponses().size() > 0) { + Map shardResponses = new HashMap<>(); + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + shardResponses.put(shardResponse.getKey(), shardResponse.getValue()); + } + allResults.get(indexName).put(shardId, shardResponses); + } else { + allResults.get(indexName).put(shardId, result.failureReason()); + } + } + for (Map.Entry> result : allResults.entrySet()) { + builder.startArray(result.getKey()); + for (Map.Entry shardResponse : result.getValue().entrySet()) { + builder.startObject(); + builder.field("shard_id", shardResponse.getKey()); + if (shardResponse.getValue() instanceof Map) { + builder.startObject("responses"); + Map results = (Map) shardResponse.getValue(); + boolean success = true; + for (Map.Entry shardCopy : results.entrySet()) { + builder.field(shardCopy.getKey().currentNodeId(), shardCopy.getValue().success() ? "success" : shardCopy.getValue().failureReason()); + if (shardCopy.getValue().success() == false) { + success = false; + } + } + builder.endObject(); + builder.field("message", success ? "success" : "failed on some copies"); + + } else { + builder.field("message", shardResponse.getValue()); // must be a string + } + builder.endObject(); + } + builder.endArray(); + } + return builder; + } +} diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java new file mode 100644 index 00000000000..3824290d0da --- /dev/null +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java @@ -0,0 +1,88 @@ +/* + * 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.seal; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.*; + +import java.io.IOException; +import java.util.Set; + +/** + */ +public class TransportSealIndicesAction extends HandledTransportAction { + + + final private SyncedFlushService syncedFlushService; + final private ClusterService clusterService; + + @Inject + public TransportSealIndicesAction(Settings settings, ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, SyncedFlushService syncedFlushService, ClusterService clusterService) { + super(settings, SealIndicesAction.NAME, threadPool, transportService, actionFilters, SealIndicesRequest.class); + this.syncedFlushService = syncedFlushService; + this.clusterService = clusterService; + } + + @Override + protected void doExecute(final SealIndicesRequest request, final ActionListener listener) { + ClusterState state = clusterService.state(); + String[] concreteIndices = state.metaData().concreteIndices(request.indicesOptions(), request.indices()); + GroupShardsIterator primaries = state.routingTable().activePrimaryShardsGrouped(concreteIndices, false); + final CountDown countDown = new CountDown(primaries.size()); + final Set results = ConcurrentCollections.newConcurrentSet(); + for (final ShardIterator shard : primaries) { + final ShardId shardId = shard.shardId(); + syncedFlushService.attemptSyncedFlush(shardId, new ActionListener() { + @Override + public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { + results.add(syncedFlushResult); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + + @Override + public void onFailure(Throwable e) { + logger.debug("{} unexpected error while executing synced flush", shardId); + results.add(new SyncedFlushService.SyncedFlushResult(shardId, e.getMessage())); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + }); + } + } +} diff --git a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java index 715d7b618a9..5dfc2da27a4 100644 --- a/src/main/java/org/elasticsearch/client/IndicesAdminClient.java +++ b/src/main/java/org/elasticsearch/client/IndicesAdminClient.java @@ -84,6 +84,9 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRespons import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequestBuilder; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequestBuilder; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateResponse; @@ -359,6 +362,27 @@ public interface IndicesAdminClient extends ElasticsearchClient { */ FlushRequestBuilder prepareFlush(String... indices); + /** + * Explicitly sync flush one or more indices + * + * @param request The seal indices request + * @return A result future + */ + ActionFuture sealIndices(SealIndicesRequest request); + + /** + * Explicitly sync flush one or more indices + * + * @param request The seal indices request + * @param listener A listener to be notified with a result + */ + void sealIndices(SealIndicesRequest request, ActionListener listener); + + /** + * Explicitly seal one or more indices + */ + SealIndicesRequestBuilder prepareSealIndices(String... indices); + /** * Explicitly optimize one or more indices into a the number of segments. * @@ -723,4 +747,5 @@ public interface IndicesAdminClient extends ElasticsearchClient { * @see #getSettings(org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest) */ GetSettingsRequestBuilder prepareGetSettings(String... indices); + } diff --git a/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/src/main/java/org/elasticsearch/client/support/AbstractClient.java index edec9af848a..f005ac22d8f 100644 --- a/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -180,6 +180,10 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequestBuilder; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest; import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequestBuilder; @@ -1323,6 +1327,21 @@ public abstract class AbstractClient extends AbstractComponent implements Client return new FlushRequestBuilder(this, FlushAction.INSTANCE).setIndices(indices); } + @Override + public ActionFuture sealIndices(SealIndicesRequest request) { + return execute(SealIndicesAction.INSTANCE, request); + } + + @Override + public void sealIndices(SealIndicesRequest request, ActionListener listener) { + execute(SealIndicesAction.INSTANCE, request, listener); + } + + @Override + public SealIndicesRequestBuilder prepareSealIndices(String... indices) { + return new SealIndicesRequestBuilder(this, SealIndicesAction.INSTANCE).indices(indices); + } + @Override public void getMappings(GetMappingsRequest request, ActionListener listener) { execute(GetMappingsAction.INSTANCE, request, listener); diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index a15ad796da8..404ef74eff7 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -18,17 +18,13 @@ */ package org.elasticsearch.indices; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.LatchedActionListener; -import org.elasticsearch.action.admin.cluster.node.liveness.LivenessRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest; -import org.elasticsearch.action.support.DelegatingActionListener; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.ImmutableShardRouting; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -39,7 +35,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.IndexService; @@ -56,10 +51,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; public class SyncedFlushService extends AbstractComponent { @@ -135,7 +128,6 @@ public class SyncedFlushService extends AbstractComponent { if (inflight != 1) { actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); } - String syncId = Strings.base64UUID(); sendSyncRequests(syncId, activeShards, state, commitIds, shardId, actionListener); } catch (Throwable t) { @@ -155,6 +147,7 @@ public class SyncedFlushService extends AbstractComponent { } final AtomicInteger result = new AtomicInteger(-1); final CountDownLatch latch = new CountDownLatch(1); + logger.trace("{} retrieving in flight operation count", shardId); transportService.sendRequest(primaryNode, IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpsRequest(shardId), new BaseTransportResponseHandler() { @Override @@ -340,21 +333,25 @@ public class SyncedFlushService extends AbstractComponent { return new InFlightOpsResponse(opCount); } - public static class SyncedFlushResult { - private final String failureReason; - private final Map shardResponses; - private final String syncId; + public static class SyncedFlushResult extends TransportResponse { + private String failureReason; + private Map shardResponses; + private String syncId; + private ShardId shardId; + + public SyncedFlushResult() { + + } public ShardId getShardId() { return shardId; } - private final ShardId shardId; - /** * failure constructor */ - SyncedFlushResult(ShardId shardId, String failureReason) { + + public SyncedFlushResult(ShardId shardId, String failureReason) { this.syncId = null; this.failureReason = failureReason; this.shardResponses = new HashMap<>(); @@ -364,7 +361,8 @@ public class SyncedFlushService extends AbstractComponent { /** * success constructor */ - SyncedFlushResult(ShardId shardId, String syncId, Map shardResponses) { + + public SyncedFlushResult(ShardId shardId, String syncId, Map shardResponses) { this.failureReason = null; this.shardResponses = shardResponses; this.syncId = syncId; @@ -404,6 +402,38 @@ public class SyncedFlushService extends AbstractComponent { return shardResponses; } + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalString(failureReason); + out.writeOptionalString(syncId); + out.writeVInt(shardResponses.size()); + for (Map.Entry result : shardResponses.entrySet()) { + result.getKey().writeTo(out); + result.getValue().writeTo(out); + } + shardId.writeTo(out); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + failureReason = in.readOptionalString(); + syncId = in.readOptionalString(); + int size = in.readVInt(); + shardResponses = new HashMap<>(); + for (int i = 0; i < size; i++) { + ImmutableShardRouting shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); + SyncedFlushResponse syncedFlushRsponse = new SyncedFlushResponse(); + syncedFlushRsponse.readFrom(in); + shardResponses.put(shardRouting, syncedFlushRsponse); + } + shardId = ShardId.readShardId(in); + } + + public ShardId shardId() { + return shardId; + } } final static class PreSyncedFlushRequest extends TransportRequest { @@ -521,7 +551,7 @@ public class SyncedFlushService extends AbstractComponent { } } - static final class SyncedFlushResponse extends TransportResponse { + public static final class SyncedFlushResponse extends TransportResponse { /** * a non null value indicates a failure to sync flush. null means success diff --git a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java index 59c28553622..a3864d5886e 100644 --- a/src/main/java/org/elasticsearch/rest/action/RestActionModule.java +++ b/src/main/java/org/elasticsearch/rest/action/RestActionModule.java @@ -23,6 +23,9 @@ import com.google.common.collect.Lists; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.multibindings.Multibinder; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.action.admin.indices.seal.RestSealIndicesAction; +import org.elasticsearch.rest.action.admin.indices.upgrade.RestUpgradeAction; +import org.elasticsearch.rest.action.admin.cluster.repositories.verify.RestVerifyRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.health.RestClusterHealthAction; import org.elasticsearch.rest.action.admin.cluster.node.hotthreads.RestNodesHotThreadsAction; import org.elasticsearch.rest.action.admin.cluster.node.info.RestNodesInfoAction; @@ -30,7 +33,6 @@ import org.elasticsearch.rest.action.admin.cluster.node.stats.RestNodesStatsActi import org.elasticsearch.rest.action.admin.cluster.repositories.delete.RestDeleteRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.repositories.get.RestGetRepositoriesAction; import org.elasticsearch.rest.action.admin.cluster.repositories.put.RestPutRepositoryAction; -import org.elasticsearch.rest.action.admin.cluster.repositories.verify.RestVerifyRepositoryAction; import org.elasticsearch.rest.action.admin.cluster.reroute.RestClusterRerouteAction; import org.elasticsearch.rest.action.admin.cluster.settings.RestClusterGetSettingsAction; import org.elasticsearch.rest.action.admin.cluster.settings.RestClusterUpdateSettingsAction; @@ -73,7 +75,6 @@ import org.elasticsearch.rest.action.admin.indices.template.delete.RestDeleteInd import org.elasticsearch.rest.action.admin.indices.template.get.RestGetIndexTemplateAction; import org.elasticsearch.rest.action.admin.indices.template.head.RestHeadIndexTemplateAction; import org.elasticsearch.rest.action.admin.indices.template.put.RestPutIndexTemplateAction; -import org.elasticsearch.rest.action.admin.indices.upgrade.RestUpgradeAction; import org.elasticsearch.rest.action.admin.indices.validate.query.RestValidateQueryAction; import org.elasticsearch.rest.action.admin.indices.warmer.delete.RestDeleteWarmerAction; import org.elasticsearch.rest.action.admin.indices.warmer.get.RestGetWarmerAction; @@ -182,6 +183,7 @@ public class RestActionModule extends AbstractModule { bind(RestRefreshAction.class).asEagerSingleton(); bind(RestFlushAction.class).asEagerSingleton(); + bind(RestSealIndicesAction.class).asEagerSingleton(); bind(RestOptimizeAction.class).asEagerSingleton(); bind(RestUpgradeAction.class).asEagerSingleton(); bind(RestClearIndicesCacheAction.class).asEagerSingleton(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java new file mode 100644 index 00000000000..e5953466f70 --- /dev/null +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java @@ -0,0 +1,67 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.admin.indices.seal; + +import org.elasticsearch.action.admin.indices.seal.SealIndicesAction; +import org.elasticsearch.action.admin.indices.seal.SealIndicesRequest; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.*; +import org.elasticsearch.rest.action.support.RestBuilderListener; + +import static org.elasticsearch.rest.RestRequest.Method.GET; +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.rest.RestStatus.OK; + +/** + * + */ +public class RestSealIndicesAction extends BaseRestHandler { + + @Inject + public RestSealIndicesAction(Settings settings, RestController controller, Client client) { + super(settings, controller, client); + controller.registerHandler(POST, "/_seal", this); + controller.registerHandler(POST, "/{index}/_seal", this); + + controller.registerHandler(GET, "/_seal", this); + controller.registerHandler(GET, "/{index}/_seal", this); + } + + @Override + public void handleRequest(final RestRequest request, final RestChannel channel, final Client client) { + String[] indices = Strings.splitStringByCommaToArray(request.param("index")); + SealIndicesRequest sealIndicesRequest = new SealIndicesRequest(indices); + client.admin().indices().execute(SealIndicesAction.INSTANCE, sealIndicesRequest, new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(SealIndicesResponse response, XContentBuilder builder) throws Exception { + builder.startObject(); + builder = response.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + return new BytesRestResponse(OK, builder); + } + }); + } +} diff --git a/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java b/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java new file mode 100644 index 00000000000..14ca12e4f57 --- /dev/null +++ b/src/test/java/org/elasticsearch/action/admin/indices/seal/SealIndicesTests.java @@ -0,0 +1,117 @@ +/* + * 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.seal; + +import org.elasticsearch.cluster.routing.ImmutableShardRouting; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.io.IOException; +import java.util.*; + +import static org.elasticsearch.test.XContentTestUtils.convertToMap; +import static org.hamcrest.Matchers.equalTo; + +public class SealIndicesTests extends ElasticsearchTestCase { + + public void testSealIndicesResponseStreaming() throws IOException { + + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult = createSyncedFlushResult(0, "test"); + shardResults.add(syncedFlushResult); + // add one result where all failed + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", 1), "all failed :("); + shardResults.add(syncedFlushResult); + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + BytesStreamOutput out = new BytesStreamOutput(); + sealIndicesResponse.writeTo(out); + out.close(); + StreamInput in = new BytesStreamInput(out.bytes()); + SealIndicesResponse readResponse = new SealIndicesResponse(); + readResponse.readFrom(in); + Map asMap = convertToMap(readResponse); + assertResponse(asMap); + } + + public void testXContentResponse() throws IOException { + + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult = createSyncedFlushResult(0, "test"); + shardResults.add(syncedFlushResult); + // add one result where all failed + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", 1), "all failed :("); + shardResults.add(syncedFlushResult); + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + Map asMap = convertToMap(sealIndicesResponse); + assertResponse(asMap); + } + + protected void assertResponse(Map asMap) { + assertNotNull(asMap.get("test")); + assertThat((Integer) (((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("shard_id")), equalTo(0)); + assertThat((String) (((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("message")), equalTo("failed on some copies")); + HashMap shardResponses = (HashMap) ((HashMap) ((ArrayList) asMap.get("test")).get(0)).get("responses"); + assertThat(shardResponses.get("node_1"), equalTo("failed for some reason")); + assertThat(shardResponses.get("node_2"), equalTo("success")); + HashMap failedShard = (HashMap) (((ArrayList) asMap.get("test")).get(1)); + assertThat((Integer) (failedShard.get("shard_id")), equalTo(1)); + assertThat((String) (failedShard.get("message")), equalTo("all failed :(")); + } + + public void testXContentResponseSortsShards() throws IOException { + Set shardResults = new HashSet<>(); + // add one result where one shard failed and one succeeded + SyncedFlushService.SyncedFlushResult syncedFlushResult; + for (int i = 100000; i >= 0; i--) { + if (randomBoolean()) { + syncedFlushResult = createSyncedFlushResult(i, "test"); + shardResults.add(syncedFlushResult); + } else { + syncedFlushResult = new SyncedFlushService.SyncedFlushResult(new ShardId("test", i), "all failed :("); + shardResults.add(syncedFlushResult); + } + } + SealIndicesResponse sealIndicesResponse = new SealIndicesResponse(shardResults); + Map asMap = convertToMap(sealIndicesResponse); + assertNotNull(asMap.get("test")); + for (int i = 0; i < 100000; i++) { + assertThat((Integer) (((HashMap) ((ArrayList) asMap.get("test")).get(i)).get("shard_id")), equalTo(i)); + } + } + + protected SyncedFlushService.SyncedFlushResult createSyncedFlushResult(int shardId, String index) { + Map responses = new HashMap<>(); + ImmutableShardRouting shardRouting = new ImmutableShardRouting(index, shardId, "node_1", false, ShardRoutingState.RELOCATING, 2); + SyncedFlushService.SyncedFlushResponse syncedFlushResponse = new SyncedFlushService.SyncedFlushResponse("failed for some reason"); + responses.put(shardRouting, syncedFlushResponse); + shardRouting = new ImmutableShardRouting(index, shardId, "node_2", false, ShardRoutingState.RELOCATING, 2); + syncedFlushResponse = new SyncedFlushService.SyncedFlushResponse(); + responses.put(shardRouting, syncedFlushResponse); + return new SyncedFlushService.SyncedFlushResult(new ShardId(index, shardId), "some_sync_id", responses); + } +} diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index 500579aeb3c..d66100218a7 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -22,7 +22,12 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; @@ -32,10 +37,14 @@ import org.junit.Test; import java.io.IOException; import java.util.Arrays; +import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import static java.lang.Thread.sleep; import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; @@ -98,8 +107,119 @@ public class FlushTest extends ElasticsearchIntegrationTest { String syncId = result.syncId(); for (ShardStats shardStats : indexStats.getShards()) { final String shardSyncId = shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID); - assertThat(shardSyncId, equalTo(syncId)); + assertThat(shardSyncId, equalTo(syncId)); } + // now, start new node and relocate a shard there and see if sync id still there + String newNodeName = internalCluster().startNode(); + ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + ShardRouting shardRouting = clusterState.getRoutingTable().index("test").shard(0).iterator().next(); + String currentNodeName = clusterState.nodes().resolveNode(shardRouting.currentNodeId()).name(); + assertFalse(currentNodeName.equals(newNodeName)); + internalCluster().client().admin().cluster().prepareReroute().add(new MoveAllocationCommand(new ShardId("test", 0), currentNodeName, newNodeName)).get(); + + client().admin().cluster().prepareHealth() + .setWaitForRelocatingShards(0) + .get(); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + + client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0).build()).get(); + ensureGreen("test"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, internalCluster().numDataNodes() - 1).build()).get(); + ensureGreen("test"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + } + + @TestLogging("indices:TRACE") + public void testSyncedFlushWithApi() throws ExecutionException, InterruptedException, IOException { + + createIndex("test"); + ensureGreen(); + + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + logger.info("--> trying sync flush"); + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + logger.info("--> sync flush done"); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + } + + @TestLogging("indices:TRACE") + public void testSyncedFlushWithApiAndConcurrentIndexing() throws Exception { + + internalCluster().ensureAtLeastNumDataNodes(3); + createIndex("test"); + + client().admin().indices().prepareUpdateSettings("test").setSettings( + ImmutableSettings.builder().put("index.translog.disable_flush", true).put("index.refresh_interval", -1).put("index.number_of_replicas", internalCluster().numDataNodes() - 1)) + .get(); + ensureGreen(); + final AtomicBoolean stop = new AtomicBoolean(false); + final AtomicInteger numDocs = new AtomicInteger(0); + Thread indexingThread = new Thread() { + @Override + public void run() { + while (stop.get() == false) { + client().prepareIndex().setIndex("test").setType("doc").setSource("{}").get(); + numDocs.incrementAndGet(); + } + } + }; + indexingThread.start(); + + IndexStats indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + } + logger.info("--> trying sync flush"); + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + logger.info("--> sync flush done"); + stop.set(true); + indexingThread.join(); + indexStats = client().admin().indices().prepareStats("test").get().getIndex("test"); + for (ShardStats shardStats : indexStats.getShards()) { + assertFlushResponseEqualsShardStats(shardStats, sealIndicesResponse); + } + refresh(); + assertThat(client().prepareCount().get().getCount(), equalTo((long) numDocs.get())); + logger.info("indexed {} docs", client().prepareCount().get().getCount()); + logClusterState(); + internalCluster().fullRestart(); + ensureGreen(); + assertThat(client().prepareCount().get().getCount(), equalTo((long) numDocs.get())); + } + + private void assertFlushResponseEqualsShardStats(ShardStats shardStats, SealIndicesResponse sealIndicesResponse) { + + for (SyncedFlushService.SyncedFlushResult shardResult : sealIndicesResponse.results()) { + if (shardStats.getShardRouting().getId() == shardResult.shardId().getId()) { + for (Map.Entry singleResponse : shardResult.shardResponses().entrySet()) { + if (singleResponse.getKey().currentNodeId().equals(shardStats.getShardRouting().currentNodeId())) { + if (singleResponse.getValue().success()) { + assertNotNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + logger.info("sync flushed {} on node {}", singleResponse.getKey().shardId(), singleResponse.getKey().currentNodeId()); + } else { + assertNull(shardStats.getCommitStats().getUserData().get(Engine.SYNC_COMMIT_ID)); + logger.info("sync flush failed for {} on node {}", singleResponse.getKey().shardId(), singleResponse.getKey().currentNodeId()); + } + } + } + } + } } } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 67991a896ea..ffed416f983 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -49,6 +49,7 @@ import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.admin.indices.optimize.OptimizeResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -1497,8 +1498,13 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase client().admin().indices().prepareRefresh(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( new LatchedActionListener(newLatch(inFlightAsyncOperations))); } else if (maybeFlush && rarely()) { - client().admin().indices().prepareFlush(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( - new LatchedActionListener(newLatch(inFlightAsyncOperations))); + if (randomBoolean()) { + client().admin().indices().prepareFlush(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute( + new LatchedActionListener(newLatch(inFlightAsyncOperations))); + } else { + client().admin().indices().prepareSealIndices(indices).execute( + new LatchedActionListener(newLatch(inFlightAsyncOperations))); + } } else if (rarely()) { client().admin().indices().prepareOptimize(indices).setIndicesOptions(IndicesOptions.lenientExpandOpen()).setMaxNumSegments(between(1, 10)).setFlush(maybeFlush && randomBoolean()).execute( new LatchedActionListener(newLatch(inFlightAsyncOperations))); From f1948cf95c6c5a9d31fff2ebc97d8a5912f5d90f Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Wed, 13 May 2015 15:12:17 +0200 Subject: [PATCH 13/38] doc for seal api and doc for syned flush in general --- docs/reference/indices/seal.asciidoc | 74 +++++++++++++++++++ .../indices/SyncedFlushService.java | 25 +++++++ 2 files changed, 99 insertions(+) create mode 100644 docs/reference/indices/seal.asciidoc diff --git a/docs/reference/indices/seal.asciidoc b/docs/reference/indices/seal.asciidoc new file mode 100644 index 00000000000..03de367af71 --- /dev/null +++ b/docs/reference/indices/seal.asciidoc @@ -0,0 +1,74 @@ +[[indices-seal]] +== Seal + +The seal API allows to flush one or more indices and adds a marker to primaries and replicas if there are no pending write operations. +The seal marker is used during recovery after a node restarts. If a replica is allocated on a node which already has a shard copy with the same seal as the primary then no files will be copied during recovery. +Sealing is a best effort operation. If write operations are ongoing while the sealing is in progress then writing the seal might fail on some copies. + +[source,js] +-------------------------------------------------- +$ curl -XPOST 'http://localhost:9200/twitter/_seal' +-------------------------------------------------- + +The response contains details about for which shards a seal was written and the reason in case of failure. +Response in case all copies of a shard successfully wrote the seal: + +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "responses": { + "5wjOIntuRqy9F_7JRrrLwA": "success", + "M2iCBe-nS5yaInE8volfSg": "success" + }, + "message": "success" + } +} +-------------------------------------------------- + + +Response in case some copies of a shard failed: +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "responses": { + "M2iCBe-nS5yaInE8volfSg": "pending operations", + "5wjOIntuRqy9F_7JRrrLwA": "success" + }, + "message": "failed on some copies" + } +} +-------------------------------------------------- + + +Response in case all copies of a shard failed: +[source,js] +-------------------------------------------------- +{ + "twitter": [ + { + "shard_id": 0, + "message": "operation counter on primary is non zero [2]" + } +} +-------------------------------------------------- + + +[float] +[[seal-multi-index]] +=== Multi Index + +The seal API can be applied to more than one index with a single call, +or even on `_all` the indices. + +[source,js] +-------------------------------------------------- +$ curl -XPOST 'http://localhost:9200/kimchy,elasticsearch/_seal' + +$ curl -XPOST 'http://localhost:9200/_seal' +-------------------------------------------------- diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 404ef74eff7..0bf274021b7 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -106,6 +106,31 @@ public class SyncedFlushService extends AbstractComponent { }); } + /* + * Tries to flush all copies of a shard and write a sync id to it. + * After a synced flush two shard copies may only contain the same sync id if they contain the same documents. + * To ensure this, synced flush works in three steps: + * 1. Flush all shard copies and gather the commit points for each copy after the flush + * 2. Ensure that there are no ongoing indexing operations on the primary + * 3. Perform an additional flush on each shard copy that writes the sync id + * + * Step 3 is only executed on a shard if + * a) the shard has no uncommitted changes since the last flush + * b) the last flush was the one executed in 1 (use the collected commit id to verify this) + * + * This alone is not enough to ensure that all copies contain the same documents. Without step 2 a sync id would be written for inconsistent copies in the following scenario: + * + * Write operation has completed on a primary and is being sent to replicas. The write request does not reach the replicas until sync flush is finished. + * Step 1 is executed. After the flush the commit points on primary contains a write operation that the replica does not have. + * Step 3 will be executed on primary and replica as well because there are no uncommitted changes on primary (the first flush committed them) and there are no uncommitted + * changes on the replica (the write operation has not reached the replica yet). + * + * Step 2 detects this scenario and fails the whole synced flush if a write operation is ongoing on the primary. + * + * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies. + * + * **/ + public void attemptSyncedFlush(ShardId shardId, ActionListener actionListener) { try { final ClusterState state = clusterService.state(); From 807b3c6b9504d33866e95072cfbcd3fa7d063b1d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 13 May 2015 16:03:53 +0200 Subject: [PATCH 14/38] [TEST] Add basic unittest like tests for SyncedFlushService This commit adds a SingleNodeTest that runs through the different fialure szenarios on a single shard to test the basic operations. --- .../action/LatchedActionListener.java | 2 +- .../indices/SyncedFlushService.java | 29 ++- .../indices/SycnedFlushSingleNodeTest.java | 228 ++++++++++++++++++ .../elasticsearch/test/SyncedFlushUtil.java | 44 ++-- 4 files changed, 272 insertions(+), 31 deletions(-) create mode 100644 src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java diff --git a/src/main/java/org/elasticsearch/action/LatchedActionListener.java b/src/main/java/org/elasticsearch/action/LatchedActionListener.java index fa17ae5c8fe..fb0fd81a7be 100644 --- a/src/main/java/org/elasticsearch/action/LatchedActionListener.java +++ b/src/main/java/org/elasticsearch/action/LatchedActionListener.java @@ -25,7 +25,7 @@ import java.util.concurrent.CountDownLatch; * An action listener that allows passing in a {@link CountDownLatch} that * will be counted down after onResponse or onFailure is called */ -public final class LatchedActionListener implements ActionListener { +public class LatchedActionListener implements ActionListener { private final ActionListener delegate; private final CountDownLatch latch; diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 0bf274021b7..79b33b9e0de 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ImmutableShardRouting; import org.elasticsearch.cluster.routing.IndexRoutingTable; @@ -134,14 +135,7 @@ public class SyncedFlushService extends AbstractComponent { public void attemptSyncedFlush(ShardId shardId, ActionListener actionListener) { try { final ClusterState state = clusterService.state(); - final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); - if (indexRoutingTable == null) { - throw new IndexMissingException(shardId.index()); - } - final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); - if (shardRoutingTable == null) { - throw new IndexShardMissingException(shardId); - } + final IndexShardRoutingTable shardRoutingTable = getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); Map commitIds = sendPreSyncRequests(activeShards, state, shardId); @@ -160,6 +154,22 @@ public class SyncedFlushService extends AbstractComponent { } } + final IndexShardRoutingTable getActiveShardRoutings(ShardId shardId, ClusterState state) { + final IndexRoutingTable indexRoutingTable = state.routingTable().index(shardId.index().name()); + if (indexRoutingTable == null) { + IndexMetaData index = state.getMetaData().index(shardId.index().getName()); + if (index != null && index.state() == IndexMetaData.State.CLOSE) { + throw new IndexClosedException(shardId.index()); + } + throw new IndexMissingException(shardId.index()); + } + final IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.id()); + if (shardRoutingTable == null) { + throw new IndexShardMissingException(shardId); + } + return shardRoutingTable; + } + /** * returns the number of inflight operations on primary. -1 upon error. */ @@ -211,7 +221,7 @@ public class SyncedFlushService extends AbstractComponent { } - private void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { + void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { final CountDown countDownLatch = new CountDown(shards.size()); final Map results = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { @@ -583,7 +593,6 @@ public class SyncedFlushService extends AbstractComponent { */ String failureReason; - public SyncedFlushResponse() { failureReason = null; } diff --git a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java new file mode 100644 index 00000000000..ea9d77b0acc --- /dev/null +++ b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java @@ -0,0 +1,228 @@ +/* + * 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; + +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ElasticsearchSingleNodeTest; +import org.elasticsearch.test.SyncedFlushUtil; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; + +import java.util.List; +import java.util.Map; + +/** + */ +public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { + + public void testModificationPreventsSealing() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + client().prepareIndex("test", "test", "2").setSource("{}").get(); + String syncId = Strings.base64UUID(); + SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("pending operations", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + + flushService.sendPreSyncRequests(activeShards, state, shardId); // pull another commit and make sure we can't seal with the old one + listener = new SyncedFlushUtil.SyncResultListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); + listener.latch.await(); + assertNull(listener.error); + syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("commit has changed", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testSingleShardSuccess() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(1, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + SyncedFlushService.SyncedFlushResponse response = syncedFlushResult.shardResponses().values().iterator().next(); + assertTrue(response.success()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testSyncFailsIfOperationIsInFlight() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + shard.incrementOperationCounter(); + try { + SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(0, syncedFlushResult.totalShards()); + assertEquals("operation counter on primary is non zero [2]", syncedFlushResult.failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } finally { + shard.decrementOperationCounter(); + } + } + + public void testSyncFailsOnIndexClosedOrMissing() throws InterruptedException { + createIndex("test"); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.attemptSyncedFlush(new ShardId("test", 1), listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("missing", listener.error.getMessage()); + + final ShardId shardId = shard.shardId(); + + client().admin().indices().prepareClose("test").get(); + listener = new SyncedFlushUtil.SyncResultListener(); + flushService.attemptSyncedFlush(shardId, listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("closed", listener.error.getMessage()); + + listener = new SyncedFlushUtil.SyncResultListener(); + flushService.attemptSyncedFlush(new ShardId("nosuchindex", 0), listener); + listener.latch.await(); + assertNotNull(listener.error); + assertNull(listener.result); + assertEquals("no such index", listener.error.getMessage()); + } + + public void testFailAfterIntermediateCommit() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + if (randomBoolean()) { + client().prepareIndex("test", "test", "2").setSource("{}").get(); + } + client().admin().indices().prepareFlush("test").setForce(true).get(); + String syncId = Strings.base64UUID(); + final SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("commit has changed", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + public void testFailWhenCommitIsMissing() throws InterruptedException { + createIndex("test"); + client().prepareIndex("test", "test", "1").setSource("{}").get(); + IndexService test = getInstanceFromNode(IndicesService.class).indexService("test"); + IndexShard shard = test.shard(0); + + SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); + final ShardId shardId = shard.shardId(); + final ClusterState state = getInstanceFromNode(ClusterService.class).state(); + final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); + final List activeShards = shardRoutingTable.activeShards(); + assertEquals("exactly one active shard", 1, activeShards.size()); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + assertEquals("exactly one commit id", 1, commitIds.size()); + commitIds.clear(); // wipe it... + String syncId = Strings.base64UUID(); + SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); + listener.latch.await(); + assertNull(listener.error); + SyncedFlushService.SyncedFlushResult syncedFlushResult = listener.result; + assertNotNull(syncedFlushResult); + assertEquals(0, syncedFlushResult.successfulShards()); + assertEquals(1, syncedFlushResult.totalShards()); + assertEquals(syncId, syncedFlushResult.syncId()); + assertNotNull(syncedFlushResult.shardResponses().get(activeShards.get(0))); + assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); + assertEquals("no commit id from pre-sync flush", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); + ElasticsearchAssertions.assertVersionSerializable(syncedFlushResult); + } + + +} diff --git a/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java b/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java index 53f1941cfff..ddf29297851 100644 --- a/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java +++ b/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java @@ -37,31 +37,35 @@ public class SyncedFlushUtil { * Blocking version of {@link SyncedFlushService#attemptSyncedFlush(ShardId, ActionListener)} */ public static SyncedFlushService.SyncedFlushResult attemptSyncedFlush(SyncedFlushService service, ShardId shardId) { - final CountDownLatch countDown = new CountDownLatch(1); - final AtomicReference result = new AtomicReference<>(); - final AtomicReference exception = new AtomicReference<>(); - service.attemptSyncedFlush(shardId, new ActionListener() { - @Override - public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { - result.compareAndSet(null, syncedFlushResult); - countDown.countDown(); - } - - @Override - public void onFailure(Throwable e) { - exception.compareAndSet(null, e); - countDown.countDown(); - } - }); + SyncResultListener listener = new SyncResultListener(); + service.attemptSyncedFlush(shardId, listener); try { - countDown.await(); + listener.latch.await(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - if (exception.get() != null) { - throw ExceptionsHelper.convertToElastic(exception.get()); + if (listener.error != null) { + throw ExceptionsHelper.convertToElastic(listener.error); + } + return listener.result; + } + + public static final class SyncResultListener implements ActionListener { + public volatile SyncedFlushService.SyncedFlushResult result; + public volatile Throwable error; + public final CountDownLatch latch = new CountDownLatch(1); + + @Override + public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { + result = syncedFlushResult; + latch.countDown(); + } + + @Override + public void onFailure(Throwable e) { + error = e; + latch.countDown(); } - return result.get(); } } From 471cd54e39e40be3342d06050d3d7cf3e4b8af5f Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Wed, 13 May 2015 17:01:20 +0200 Subject: [PATCH 15/38] fix api hangs if no shards allocated --- .../seal/TransportSealIndicesAction.java | 7 +++- .../org/elasticsearch/indices/SealTests.java | 38 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) create mode 100644 src/test/java/org/elasticsearch/indices/SealTests.java diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java index 3824290d0da..b40be67f131 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java @@ -61,8 +61,13 @@ public class TransportSealIndicesAction extends HandledTransportAction results = ConcurrentCollections.newConcurrentSet(); + if (primaries.size() == 0) { + // no active primary available + listener.onResponse(new SealIndicesResponse(results)); + return; + } + final CountDown countDown = new CountDown(primaries.size()); for (final ShardIterator shard : primaries) { final ShardId shardId = shard.shardId(); syncedFlushService.attemptSyncedFlush(shardId, new ActionListener() { diff --git a/src/test/java/org/elasticsearch/indices/SealTests.java b/src/test/java/org/elasticsearch/indices/SealTests.java new file mode 100644 index 00000000000..346ed599635 --- /dev/null +++ b/src/test/java/org/elasticsearch/indices/SealTests.java @@ -0,0 +1,38 @@ +/* + * 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; + +import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import static org.hamcrest.Matchers.equalTo; + +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE, numDataNodes = 1) +public class SealTests extends ElasticsearchIntegrationTest { + + @Test + public void testUnallocatedShardsDoesNotHang() throws InterruptedException { + // create an index but because no data nodes are available no shards will be allocated + createIndex("test"); + // this should not hang but instead immediately return with empty result set + SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + assertThat(sealIndicesResponse.results().size(), equalTo(0)); + } +} From 3a9c2bd63969a390aa682f8305559aeefbded53c Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Wed, 13 May 2015 18:05:32 +0200 Subject: [PATCH 16/38] fix test --- .../java/org/elasticsearch/indices/SealTests.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/test/java/org/elasticsearch/indices/SealTests.java b/src/test/java/org/elasticsearch/indices/SealTests.java index 346ed599635..7da9a15da35 100644 --- a/src/test/java/org/elasticsearch/indices/SealTests.java +++ b/src/test/java/org/elasticsearch/indices/SealTests.java @@ -19,20 +19,28 @@ package org.elasticsearch.indices; import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; +import static java.lang.Thread.sleep; import static org.hamcrest.Matchers.equalTo; -@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE, numDataNodes = 1) +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) public class SealTests extends ElasticsearchIntegrationTest { @Test public void testUnallocatedShardsDoesNotHang() throws InterruptedException { - // create an index but because no data nodes are available no shards will be allocated + ImmutableSettings.Builder settingsBuilder = ImmutableSettings.builder() + .put("node.data", false) + .put("node.master", true) + .put("path.data", createTempDir().toString()); + internalCluster().startNode(settingsBuilder.build()); + // create an index but because no data nodes are available no shards will be allocated createIndex("test"); // this should not hang but instead immediately return with empty result set SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); + // just to make sure the test actually tests the right thing assertThat(sealIndicesResponse.results().size(), equalTo(0)); } } From 5eafc9198f3f7c7bf2a3aa3e6022a06815fc7aa3 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Wed, 13 May 2015 18:06:59 +0200 Subject: [PATCH 17/38] rename as requested by Luca --- .../action/admin/indices/seal/SealIndicesAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java index e2f230defaf..fbb01b05abe 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesAction.java @@ -27,7 +27,7 @@ import org.elasticsearch.client.ElasticsearchClient; public class SealIndicesAction extends Action { public static final SealIndicesAction INSTANCE = new SealIndicesAction(); - public static final String NAME = "indices:admin/sealindices"; + public static final String NAME = "indices:admin/seal"; private SealIndicesAction() { super(NAME); From 36e6718bf54bceb22b3eb807a66edfbc40e269af Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Fri, 15 May 2015 10:35:15 +0200 Subject: [PATCH 18/38] some more docs and cleanup --- .../indices/SyncedFlushService.java | 44 ++++++++++++++----- 1 file changed, 34 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 79b33b9e0de..2974b55ae72 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.indices; +import com.google.common.collect.ImmutableMap; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -57,7 +58,6 @@ import java.util.concurrent.atomic.AtomicInteger; public class SyncedFlushService extends AbstractComponent { - // nocommmit: check these are ok public static final String PRE_SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/pre"; public static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; public static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; @@ -91,7 +91,7 @@ public class SyncedFlushService extends AbstractComponent { public void onShardInactive(final IndexShard indexShard) { // we only want to call sync flush once, so only trigger it when we are on a primary if (indexShard.routingEntry().primary()) { - attemptSyncedFlush(indexShard.shardId(), new ActionListener() { + attemptSyncedFlush(indexShard.shardId(), new ActionListener() { @Override public void onResponse(SyncedFlushResult syncedFlushResult) { logger.debug("{} sync flush on inactive shard returned successfully for sync_id: {}", syncedFlushResult.getShardId(), syncedFlushResult.syncId()); @@ -368,6 +368,9 @@ public class SyncedFlushService extends AbstractComponent { return new InFlightOpsResponse(opCount); } + /** + * Result for all copies of a shard + */ public static class SyncedFlushResult extends TransportResponse { private String failureReason; private Map shardResponses; @@ -375,7 +378,6 @@ public class SyncedFlushService extends AbstractComponent { private ShardId shardId; public SyncedFlushResult() { - } public ShardId getShardId() { @@ -385,29 +387,34 @@ public class SyncedFlushService extends AbstractComponent { /** * failure constructor */ - public SyncedFlushResult(ShardId shardId, String failureReason) { this.syncId = null; this.failureReason = failureReason; - this.shardResponses = new HashMap<>(); + this.shardResponses = ImmutableMap.of(); this.shardId = shardId; } /** * success constructor */ - public SyncedFlushResult(ShardId shardId, String syncId, Map shardResponses) { this.failureReason = null; - this.shardResponses = shardResponses; + ImmutableMap.Builder builder = ImmutableMap.builder(); + this.shardResponses = builder.putAll(shardResponses).build(); this.syncId = syncId; this.shardId = shardId; } + /** + * @return true if one or more shard copies was successful, false if all failed before step three of synced flush + */ public boolean success() { return syncId != null; } + /** + * @return the reason for the failure if synced flush failed before step three of synced flush + */ public String failureReason() { return failureReason; } @@ -417,12 +424,15 @@ public class SyncedFlushService extends AbstractComponent { } /** - * total number of shards for which a sync attempt was made + * @return total number of shards for which a sync attempt was made */ public int totalShards() { return shardResponses.size(); } + /** + * @return total number of successful shards + */ public int successfulShards() { int i = 0; for (SyncedFlushResponse result : shardResponses.values()) { @@ -433,6 +443,10 @@ public class SyncedFlushService extends AbstractComponent { return i; } + /** + * @return Individual responses for each shard copy with a detailed failure message if the copy failed to perform the synced flush. + * Empty if synced flush failed before step three. + */ public Map shardResponses() { return shardResponses; } @@ -456,13 +470,14 @@ public class SyncedFlushService extends AbstractComponent { failureReason = in.readOptionalString(); syncId = in.readOptionalString(); int size = in.readVInt(); - shardResponses = new HashMap<>(); + ImmutableMap.Builder builder = ImmutableMap.builder(); for (int i = 0; i < size; i++) { ImmutableShardRouting shardRouting = ImmutableShardRouting.readShardRoutingEntry(in); SyncedFlushResponse syncedFlushRsponse = new SyncedFlushResponse(); syncedFlushRsponse.readFrom(in); - shardResponses.put(shardRouting, syncedFlushRsponse); + builder.put(shardRouting, syncedFlushRsponse); } + shardResponses = builder.build(); shardId = ShardId.readShardId(in); } @@ -506,6 +521,9 @@ public class SyncedFlushService extends AbstractComponent { } } + /** + * Response for first step of synced flush (flush) for one shard copy + */ final static class PreSyncedFlushResponse extends TransportResponse { private byte[] commitId; @@ -586,6 +604,9 @@ public class SyncedFlushService extends AbstractComponent { } } + /** + * Response for third step of synced flush (writing the sync id) for one shard copy + */ public static final class SyncedFlushResponse extends TransportResponse { /** @@ -666,6 +687,9 @@ public class SyncedFlushService extends AbstractComponent { } } + /** + * Response for second step of synced flush (check operations in flight) + */ static final class InFlightOpsResponse extends TransportResponse { int opCount; From 81e3d5cdcb44a90c9ab197c192ee8d1c84892db3 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 14:24:15 +0200 Subject: [PATCH 19/38] use imutable object for commit id --- .../elasticsearch/index/engine/Engine.java | 60 ++++++++++++++++++- .../index/engine/InternalEngine.java | 14 ++--- .../index/engine/ShadowEngine.java | 8 +-- .../elasticsearch/index/shard/IndexShard.java | 9 ++- .../indices/SyncedFlushService.java | 39 ++++++------ .../index/engine/InternalEngineTests.java | 13 ++-- .../indices/SycnedFlushSingleNodeTest.java | 7 ++- 7 files changed, 102 insertions(+), 48 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 09a388ec532..ed2b49f303f 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -37,8 +37,12 @@ import org.apache.lucene.search.join.BitDocIdSetFilter; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountables; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.Base64; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; @@ -227,7 +231,7 @@ public abstract class Engine implements Closeable { * @param expectedCommitId the expected value of * @return true if the sync commit was made, false o.w. */ - public abstract SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException; + public abstract SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) throws EngineException; public enum SyncedFlushResult { SUCCESS, @@ -458,7 +462,7 @@ public abstract class Engine implements Closeable { * Otherwise this call will return without blocking. * @return the commit Id for the resulting commit */ - public abstract byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException; + public abstract CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException; /** * Flushes the state of the engine including the transaction log, clearing memory and persisting @@ -468,7 +472,7 @@ public abstract class Engine implements Closeable { * * @return the commit Id for the resulting commit */ - public abstract byte[] flush() throws EngineException; + public abstract CommitId flush() throws EngineException; /** * Optimizes to 1 segment @@ -1141,4 +1145,54 @@ public abstract class Engine implements Closeable { * @return */ public abstract boolean hasUncommittedChanges(); + + public static class CommitId implements Writeable { + private byte[] id; + public CommitId(byte[] id) { + assert id != null; + this.id = Arrays.copyOf(id, id.length); + } + + public CommitId(StreamInput in) throws IOException { + assert in != null; + this.id = in.readByteArray(); + } + + @Override + public String toString() { + return Base64.encodeBytes(id); + } + + @Override + public CommitId readFrom(StreamInput in) throws IOException { + byte[] bytes = in.readByteArray(); + return new CommitId(bytes); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeByteArray(id); + } + + public boolean idsEqual(byte[] id) { + return Arrays.equals(id, this.id); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + CommitId commitId = (CommitId) o; + + if (!Arrays.equals(id, commitId.id)) return false; + + return true; + } + + @Override + public int hashCode() { + return Arrays.hashCode(id); + } + } } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 70e31700c7f..3cb8999dc28 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -665,14 +665,14 @@ public class InternalEngine extends Engine { } @Override - public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) throws EngineException { + public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) throws EngineException { // best effort attempt before we acquire locks ensureOpen(); if (indexWriter.hasUncommittedChanges()) { logger.trace("can't sync commit [{}]. have pending changes", syncId); return SyncedFlushResult.FAILED_PENDING_OPERATIONS; } - if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { + if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); return SyncedFlushResult.FAILED_COMMIT_MISMATCH; } @@ -682,7 +682,7 @@ public class InternalEngine extends Engine { logger.trace("can't sync commit [{}]. have pending changes", syncId); return SyncedFlushResult.FAILED_PENDING_OPERATIONS; } - if (Arrays.equals(expectedCommitId, lastCommittedSegmentInfos.getId()) == false) { + if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); return SyncedFlushResult.FAILED_COMMIT_MISMATCH; } @@ -699,16 +699,16 @@ public class InternalEngine extends Engine { } @Override - public byte[] flush() throws EngineException { + public CommitId flush() throws EngineException { return flush(true, false, false); } @Override - public byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException { + public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { return flush(true, force, waitIfOngoing); } - private byte[] flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException { + private CommitId flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException { ensureOpen(); final byte[] newCommitId; /* @@ -799,7 +799,7 @@ public class InternalEngine extends Engine { if (engineConfig.isEnableGcDeletes()) { pruneDeletedTombstones(); } - return newCommitId; + return new CommitId(newCommitId); } private void pruneDeletedTombstones() { diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 303426d16b3..9b295e7e60f 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -125,17 +125,17 @@ public class ShadowEngine extends Engine { } @Override - public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) { throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } @Override - public byte[] flush() throws EngineException { + public CommitId flush() throws EngineException { return flush(false, false); } @Override - public byte[] flush(boolean force, boolean waitIfOngoing) throws EngineException { + public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.trace("skipping FLUSH on shadow engine"); // reread the last committed segment infos refresh("flush"); @@ -159,7 +159,7 @@ public class ShadowEngine extends Engine { } finally { store.decRef(); } - return lastCommittedSegmentInfos.getId(); + return new CommitId(lastCommittedSegmentInfos.getId()); } @Override diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 873f93140ac..fa0a8daccc2 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -115,8 +115,6 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.io.PrintStream; import java.nio.channels.ClosedByInterruptException; -import java.util.Arrays; -import java.util.Locale; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ScheduledFuture; @@ -693,13 +691,13 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } - public Engine.SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, byte[] expectedCommitId) { + public Engine.SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, Engine.CommitId expectedCommitId) { verifyStartedOrRecovering(); logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); return engine().syncFlushIfNoPendingChanges(syncId, expectedCommitId); } - public byte[] flush(FlushRequest request) throws ElasticsearchException { + public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException { boolean waitIfOngoing = request.waitIfOngoing(); boolean force = request.force(); if (logger.isTraceEnabled()) { @@ -711,7 +709,7 @@ public class IndexShard extends AbstractIndexShardComponent { verifyStartedOrRecovering(); long time = System.nanoTime(); - byte[] commitId = engine().flush(force, waitIfOngoing); + Engine.CommitId commitId = engine().flush(force, waitIfOngoing); flushMetric.inc(System.nanoTime() - time); return commitId; @@ -1385,4 +1383,5 @@ public class IndexShard extends AbstractIndexShardComponent { public Translog.Durabilty getTranslogDurability() { return engine().getTranslog().getDurabilty(); } + } diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 2974b55ae72..da7d7282555 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -49,7 +49,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.*; import java.io.IOException; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; @@ -137,7 +136,7 @@ public class SyncedFlushService extends AbstractComponent { final ClusterState state = clusterService.state(); final IndexShardRoutingTable shardRoutingTable = getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); - Map commitIds = sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = sendPreSyncRequests(activeShards, state, shardId); if (commitIds.isEmpty()) { actionListener.onResponse(new SyncedFlushResult(shardId, "all shards failed to commit on pre-sync")); @@ -221,7 +220,7 @@ public class SyncedFlushService extends AbstractComponent { } - void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { + void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { final CountDown countDownLatch = new CountDown(shards.size()); final Map results = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { @@ -234,7 +233,7 @@ public class SyncedFlushService extends AbstractComponent { } continue; } - final byte[] expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); + final Engine.CommitId expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); if (expectedCommitId == null) { logger.trace("{} can't resolve expected commit id for {}, skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); results.put(shard, new SyncedFlushResponse("no commit id from pre-sync flush")); @@ -282,9 +281,9 @@ public class SyncedFlushService extends AbstractComponent { /** * send presync requests to all started copies of the given shard */ - Map sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId) { + Map sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId) { final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); - final Map commitIds = ConcurrentCollections.newConcurrentMap(); + final Map commitIds = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { logger.trace("{} sending pre-synced flush request to {}", shardId, shard); final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); @@ -301,7 +300,7 @@ public class SyncedFlushService extends AbstractComponent { @Override public void handleResponse(PreSyncedFlushResponse response) { - byte[] existing = commitIds.put(node.id(), response.commitId()); + Engine.CommitId existing = commitIds.put(node.id(), response.commitId()); assert existing == null : "got two answers for node [" + node + "]"; // count after the assert so we won't decrement twice in handleException countDownLatch.countDown(); @@ -334,9 +333,9 @@ public class SyncedFlushService extends AbstractComponent { IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); FlushRequest flushRequest = new FlushRequest().force(false).waitIfOngoing(true); logger.trace("{} performing pre sync flush", request.shardId()); - byte[] id = indexShard.flush(flushRequest); - logger.trace("{} pre sync flush done. commit id {}", request.shardId(), id); - return new PreSyncedFlushResponse(id); + Engine.CommitId commitId = indexShard.flush(flushRequest); + logger.trace("{} pre sync flush done. commit id {}", request.shardId(), commitId); + return new PreSyncedFlushResponse(commitId); } private SyncedFlushResponse performSyncedFlush(SyncedFlushRequest request) { @@ -526,42 +525,42 @@ public class SyncedFlushService extends AbstractComponent { */ final static class PreSyncedFlushResponse extends TransportResponse { - private byte[] commitId; + Engine.CommitId commitId; PreSyncedFlushResponse() { } - PreSyncedFlushResponse(byte[] commitId) { + PreSyncedFlushResponse(Engine.CommitId commitId) { this.commitId = commitId; } - public byte[] commitId() { + public Engine.CommitId commitId() { return commitId; } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - commitId = in.readByteArray(); + commitId = new Engine.CommitId(in); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeByteArray(commitId); + commitId.writeTo(out); } } static final class SyncedFlushRequest extends TransportRequest { private String syncId; - private byte[] expectedCommitId; + private Engine.CommitId expectedCommitId; private ShardId shardId; public SyncedFlushRequest() { } - public SyncedFlushRequest(ShardId shardId, String syncId, byte[] expectedCommitId) { + public SyncedFlushRequest(ShardId shardId, String syncId, Engine.CommitId expectedCommitId) { this.expectedCommitId = expectedCommitId; this.shardId = shardId; this.syncId = syncId; @@ -571,7 +570,7 @@ public class SyncedFlushService extends AbstractComponent { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); shardId = ShardId.readShardId(in); - expectedCommitId = in.readByteArray(); + expectedCommitId = new Engine.CommitId(in); syncId = in.readString(); } @@ -579,7 +578,7 @@ public class SyncedFlushService extends AbstractComponent { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); shardId.writeTo(out); - out.writeByteArray(expectedCommitId); + expectedCommitId.writeTo(out); out.writeString(syncId); } @@ -591,7 +590,7 @@ public class SyncedFlushService extends AbstractComponent { return syncId; } - public byte[] expectedCommitId() { + public Engine.CommitId expectedCommitId() { return expectedCommitId; } diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index fabbfb49f8c..9788e23b4d9 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -41,6 +41,7 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.Base64; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -690,11 +691,12 @@ public class InternalEngineTests extends ElasticsearchTestCase { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); engine.create(new Engine.Create(null, newUid("1"), doc)); - byte[] commitID = engine.flush(); - assertThat(commitID, equalTo(store.readLastCommittedSegmentsInfo().getId())); - byte[] fakeId = commitID.clone(); - fakeId[0] = (byte) ~fakeId[0]; - assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", fakeId), + Engine.CommitId commitID = engine.flush(); + assertThat(commitID, equalTo(new Engine.CommitId(store.readLastCommittedSegmentsInfo().getId()))); + byte[] wrongBytes = Base64.decode(commitID.toString()); + wrongBytes[0] = (byte) ~wrongBytes[0]; + Engine.CommitId wrongId = new Engine.CommitId(wrongBytes); + assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", wrongId), equalTo(Engine.SyncedFlushResult.FAILED_COMMIT_MISMATCH)); engine.create(new Engine.Create(null, newUid("2"), doc)); assertThat("should fail to sync flush with right id but pending doc", engine.syncFlushIfNoPendingChanges(syncId + "2", commitID), @@ -1797,5 +1799,4 @@ public class InternalEngineTests extends ElasticsearchTestCase { recoveredOps.incrementAndGet(); } } - } diff --git a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java index ea9d77b0acc..beb55f6ff4e 100644 --- a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java +++ b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Strings; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchSingleNodeTest; @@ -49,7 +50,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); client().prepareIndex("test", "test", "2").setSource("{}").get(); String syncId = Strings.base64UUID(); @@ -171,7 +172,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); if (randomBoolean()) { client().prepareIndex("test", "test", "2").setSource("{}").get(); @@ -205,7 +206,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); commitIds.clear(); // wipe it... String syncId = Strings.base64UUID(); From 018ad0bd76cc99c617ba51b3a56d5c08d95d7ef8 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 14:32:33 +0200 Subject: [PATCH 20/38] final --- .../action/admin/indices/seal/SealIndicesResponse.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java index c4e895b5cc8..160f28efe63 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java @@ -35,9 +35,10 @@ import java.util.*; */ public class SealIndicesResponse extends ActionResponse implements ToXContent { - private Set results; + final private Set results; SealIndicesResponse() { + results = new HashSet<>(); } SealIndicesResponse(Set results) { @@ -48,7 +49,7 @@ public class SealIndicesResponse extends ActionResponse implements ToXContent { public void readFrom(StreamInput in) throws IOException { super.readFrom(in); int size = in.readVInt(); - results = new HashSet<>(); + results.clear(); for (int i = 0; i < size; i++) { SyncedFlushService.SyncedFlushResult syncedFlushResult = new SyncedFlushService.SyncedFlushResult(); syncedFlushResult.readFrom(in); From 5ac07481eb2c65283d76da08ec7a927ba61b3ee0 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 14:43:27 +0200 Subject: [PATCH 21/38] assign variables --- .../org/elasticsearch/gateway/GatewayAllocator.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index edd30b92909..9f326c7032f 100644 --- a/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -331,17 +331,18 @@ public class GatewayAllocator extends AbstractComponent { if (primaryNodeStore != null && primaryNodeStore.allocated()) { long sizeMatched = 0; + String primarySyncId = primaryNodeStore.syncId(); + String replicaSyncId = storeFilesMetaData.syncId(); // see if we have a sync id we can make use of - if (storeFilesMetaData.syncId() != null && storeFilesMetaData.syncId().equals(primaryNodeStore.syncId())) { - logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), storeFilesMetaData.syncId()); + if (replicaSyncId != null && replicaSyncId.equals(primarySyncId)) { + logger.trace("{}: node [{}] has same sync id {} as primary", shard, discoNode.name(), replicaSyncId); lastNodeMatched = node; lastSizeMatched = Long.MAX_VALUE; lastDiscoNodeMatched = discoNode; } else { for (StoreFileMetaData storeFileMetaData : storeFilesMetaData) { - logger.trace("{}: node [{}] has file {}", - shard, discoNode.name(), storeFileMetaData.name()); - if (primaryNodeStore.fileExists(storeFileMetaData.name()) && primaryNodeStore.file(storeFileMetaData.name()).isSame(storeFileMetaData)) { + String metaDataFileName = storeFileMetaData.name(); + if (primaryNodeStore.fileExists(metaDataFileName) && primaryNodeStore.file(metaDataFileName).isSame(storeFileMetaData)) { sizeMatched += storeFileMetaData.length(); } } From fcea10113c052427c803ddeef651c3760da18327 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 14:46:28 +0200 Subject: [PATCH 22/38] remove impl details from method name --- src/main/java/org/elasticsearch/index/engine/Engine.java | 2 +- .../java/org/elasticsearch/index/engine/InternalEngine.java | 2 +- .../java/org/elasticsearch/index/engine/ShadowEngine.java | 2 +- src/main/java/org/elasticsearch/index/shard/IndexShard.java | 4 ++-- .../java/org/elasticsearch/indices/SyncedFlushService.java | 2 +- .../org/elasticsearch/index/engine/InternalEngineTests.java | 6 +++--- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index ed2b49f303f..29dfac0bd64 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -231,7 +231,7 @@ public abstract class Engine implements Closeable { * @param expectedCommitId the expected value of * @return true if the sync commit was made, false o.w. */ - public abstract SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) throws EngineException; + public abstract SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException; public enum SyncedFlushResult { SUCCESS, diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3cb8999dc28..b19ca927f5f 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -665,7 +665,7 @@ public class InternalEngine extends Engine { } @Override - public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) throws EngineException { + public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) throws EngineException { // best effort attempt before we acquire locks ensureOpen(); if (indexWriter.hasUncommittedChanges()) { diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 9b295e7e60f..e09acaa0e28 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -125,7 +125,7 @@ public class ShadowEngine extends Engine { } @Override - public SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, CommitId expectedCommitId) { + public SyncedFlushResult syncFlush(String syncId, CommitId expectedCommitId) { throw new UnsupportedOperationException(shardId + " sync commit operation not allowed on shadow engine"); } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fa0a8daccc2..ec29d8bb2e0 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -691,10 +691,10 @@ public class IndexShard extends AbstractIndexShardComponent { return completionStats; } - public Engine.SyncedFlushResult syncFlushIfNoPendingChanges(String syncId, Engine.CommitId expectedCommitId) { + public Engine.SyncedFlushResult syncFlush(String syncId, Engine.CommitId expectedCommitId) { verifyStartedOrRecovering(); logger.trace("trying to sync flush. sync id [{}]. expected commit id [{}]]", syncId, expectedCommitId); - return engine().syncFlushIfNoPendingChanges(syncId, expectedCommitId); + return engine().syncFlush(syncId, expectedCommitId); } public Engine.CommitId flush(FlushRequest request) throws ElasticsearchException { diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index da7d7282555..45a20745c09 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -342,7 +342,7 @@ public class SyncedFlushService extends AbstractComponent { IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexShard indexShard = indexService.shardSafe(request.shardId().id()); logger.trace("{} performing sync flush. sync id [{}], expected commit id {}", request.shardId(), request.syncId(), request.expectedCommitId()); - Engine.SyncedFlushResult result = indexShard.syncFlushIfNoPendingChanges(request.syncId(), request.expectedCommitId()); + Engine.SyncedFlushResult result = indexShard.syncFlush(request.syncId(), request.expectedCommitId()); logger.trace("{} sync flush done. sync id [{}], result [{}]", request.shardId(), request.syncId(), result); switch (result) { case SUCCESS: diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 9788e23b4d9..f87b59b42ee 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -696,13 +696,13 @@ public class InternalEngineTests extends ElasticsearchTestCase { byte[] wrongBytes = Base64.decode(commitID.toString()); wrongBytes[0] = (byte) ~wrongBytes[0]; Engine.CommitId wrongId = new Engine.CommitId(wrongBytes); - assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlushIfNoPendingChanges(syncId + "1", wrongId), + assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlush(syncId + "1", wrongId), equalTo(Engine.SyncedFlushResult.FAILED_COMMIT_MISMATCH)); engine.create(new Engine.Create(null, newUid("2"), doc)); - assertThat("should fail to sync flush with right id but pending doc", engine.syncFlushIfNoPendingChanges(syncId + "2", commitID), + assertThat("should fail to sync flush with right id but pending doc", engine.syncFlush(syncId + "2", commitID), equalTo(Engine.SyncedFlushResult.FAILED_PENDING_OPERATIONS)); commitID = engine.flush(); - assertThat("should succeed to flush commit with right id and no pending doc", engine.syncFlushIfNoPendingChanges(syncId, commitID), + assertThat("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), equalTo(Engine.SyncedFlushResult.SUCCESS)); assertThat(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); assertThat(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); From a0315b5cb8c1a0a8b09e4a95e3fef12e5a49273f Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 14:47:31 +0200 Subject: [PATCH 23/38] remove FAILED suffix --- src/main/java/org/elasticsearch/index/engine/Engine.java | 4 ++-- .../org/elasticsearch/index/engine/InternalEngine.java | 8 ++++---- .../org/elasticsearch/indices/SyncedFlushService.java | 4 ++-- .../elasticsearch/index/engine/InternalEngineTests.java | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 29dfac0bd64..1fa12398f21 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -235,8 +235,8 @@ public abstract class Engine implements Closeable { public enum SyncedFlushResult { SUCCESS, - FAILED_COMMIT_MISMATCH, - FAILED_PENDING_OPERATIONS + COMMIT_MISMATCH, + PENDING_OPERATIONS } final protected GetResult getFromSearcher(Get get) throws EngineException { diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index b19ca927f5f..1792051b152 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -670,21 +670,21 @@ public class InternalEngine extends Engine { ensureOpen(); if (indexWriter.hasUncommittedChanges()) { logger.trace("can't sync commit [{}]. have pending changes", syncId); - return SyncedFlushResult.FAILED_PENDING_OPERATIONS; + return SyncedFlushResult.PENDING_OPERATIONS; } if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); - return SyncedFlushResult.FAILED_COMMIT_MISMATCH; + return SyncedFlushResult.COMMIT_MISMATCH; } try (ReleasableLock lock = writeLock.acquire()) { ensureOpen(); if (indexWriter.hasUncommittedChanges()) { logger.trace("can't sync commit [{}]. have pending changes", syncId); - return SyncedFlushResult.FAILED_PENDING_OPERATIONS; + return SyncedFlushResult.PENDING_OPERATIONS; } if (expectedCommitId.idsEqual(lastCommittedSegmentInfos.getId()) == false) { logger.trace("can't sync commit [{}]. current commit id is not equal to expected.", syncId); - return SyncedFlushResult.FAILED_COMMIT_MISMATCH; + return SyncedFlushResult.COMMIT_MISMATCH; } logger.trace("starting sync commit [{}]", syncId); final long translogId = translog.currentId(); diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 45a20745c09..5a5d4a42ebb 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -347,9 +347,9 @@ public class SyncedFlushService extends AbstractComponent { switch (result) { case SUCCESS: return new SyncedFlushResponse(); - case FAILED_COMMIT_MISMATCH: + case COMMIT_MISMATCH: return new SyncedFlushResponse("commit has changed"); - case FAILED_PENDING_OPERATIONS: + case PENDING_OPERATIONS: return new SyncedFlushResponse("pending operations"); default: throw new ElasticsearchException("unknown synced flush result [" + result + "]"); diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f87b59b42ee..9c7695847a5 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -697,10 +697,10 @@ public class InternalEngineTests extends ElasticsearchTestCase { wrongBytes[0] = (byte) ~wrongBytes[0]; Engine.CommitId wrongId = new Engine.CommitId(wrongBytes); assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlush(syncId + "1", wrongId), - equalTo(Engine.SyncedFlushResult.FAILED_COMMIT_MISMATCH)); + equalTo(Engine.SyncedFlushResult.COMMIT_MISMATCH)); engine.create(new Engine.Create(null, newUid("2"), doc)); assertThat("should fail to sync flush with right id but pending doc", engine.syncFlush(syncId + "2", commitID), - equalTo(Engine.SyncedFlushResult.FAILED_PENDING_OPERATIONS)); + equalTo(Engine.SyncedFlushResult.PENDING_OPERATIONS)); commitID = engine.flush(); assertThat("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), equalTo(Engine.SyncedFlushResult.SUCCESS)); From 771763ae58a3c6154d275928cb46acce28673233 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 15:40:23 +0200 Subject: [PATCH 24/38] add error for inactive primary --- .../seal/TransportSealIndicesAction.java | 57 +++++++++++-------- .../org/elasticsearch/indices/SealTests.java | 5 +- 2 files changed, 36 insertions(+), 26 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java index b40be67f131..61d5e53c32f 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/TransportSealIndicesAction.java @@ -34,11 +34,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.SyncedFlushService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.*; import java.io.IOException; +import java.util.HashSet; import java.util.Set; /** @@ -60,34 +62,39 @@ public class TransportSealIndicesAction extends HandledTransportAction listener) { ClusterState state = clusterService.state(); String[] concreteIndices = state.metaData().concreteIndices(request.indicesOptions(), request.indices()); - GroupShardsIterator primaries = state.routingTable().activePrimaryShardsGrouped(concreteIndices, false); + GroupShardsIterator primaries = state.routingTable().activePrimaryShardsGrouped(concreteIndices, true); final Set results = ConcurrentCollections.newConcurrentSet(); - if (primaries.size() == 0) { - // no active primary available - listener.onResponse(new SealIndicesResponse(results)); - return; - } - final CountDown countDown = new CountDown(primaries.size()); - for (final ShardIterator shard : primaries) { - final ShardId shardId = shard.shardId(); - syncedFlushService.attemptSyncedFlush(shardId, new ActionListener() { - @Override - public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { - results.add(syncedFlushResult); - if (countDown.countDown()) { - listener.onResponse(new SealIndicesResponse(results)); - } - } - @Override - public void onFailure(Throwable e) { - logger.debug("{} unexpected error while executing synced flush", shardId); - results.add(new SyncedFlushService.SyncedFlushResult(shardId, e.getMessage())); - if (countDown.countDown()) { - listener.onResponse(new SealIndicesResponse(results)); - } + final CountDown countDown = new CountDown(primaries.size()); + + for (final ShardIterator shard : primaries) { + if (shard.size() == 0) { + results.add(new SyncedFlushService.SyncedFlushResult(shard.shardId(), "no active primary available")); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); } - }); + } else { + final ShardId shardId = shard.shardId(); + syncedFlushService.attemptSyncedFlush(shardId, new ActionListener() { + @Override + public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { + results.add(syncedFlushResult); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + + @Override + public void onFailure(Throwable e) { + logger.debug("{} unexpected error while executing synced flush", shardId); + results.add(new SyncedFlushService.SyncedFlushResult(shardId, e.getMessage())); + if (countDown.countDown()) { + listener.onResponse(new SealIndicesResponse(results)); + } + } + }); + } } + } } diff --git a/src/test/java/org/elasticsearch/indices/SealTests.java b/src/test/java/org/elasticsearch/indices/SealTests.java index 7da9a15da35..723f699d8cd 100644 --- a/src/test/java/org/elasticsearch/indices/SealTests.java +++ b/src/test/java/org/elasticsearch/indices/SealTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices; import org.elasticsearch.action.admin.indices.seal.SealIndicesResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; @@ -41,6 +42,8 @@ public class SealTests extends ElasticsearchIntegrationTest { // this should not hang but instead immediately return with empty result set SealIndicesResponse sealIndicesResponse = client().admin().indices().prepareSealIndices("test").get(); // just to make sure the test actually tests the right thing - assertThat(sealIndicesResponse.results().size(), equalTo(0)); + int numShards = client().admin().indices().prepareGetSettings("test").get().getIndexToSettings().get("test").getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, -1); + assertThat(sealIndicesResponse.results().size(), equalTo(numShards)); + assertThat(sealIndicesResponse.results().iterator().next().failureReason(), equalTo("no active primary available")); } } From 9ca17f834b06e9495470b983153d7bf8de45729a Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 21:42:27 +0200 Subject: [PATCH 25/38] inherit from BroadcastOperationRequest --- .../indices/seal/SealIndicesRequest.java | 58 ++----------------- 1 file changed, 4 insertions(+), 54 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java index e8f032ea7ae..42cdc51ed32 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesRequest.java @@ -19,24 +19,14 @@ package org.elasticsearch.action.admin.indices.seal; -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest; -import java.io.IOException; import java.util.Arrays; /** * A request to seal one or more indices. */ -public class SealIndicesRequest extends ActionRequest implements IndicesRequest.Replaceable { - - private String[] indices; - - private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed(); +public class SealIndicesRequest extends BroadcastOperationRequest { SealIndicesRequest() { } @@ -46,54 +36,14 @@ public class SealIndicesRequest extends ActionRequest implements IndicesRequest. * be sealed. */ public SealIndicesRequest(String... indices) { - this.indices = indices; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeStringArrayNullable(indices); - indicesOptions.writeIndicesOptions(out); + super(indices); } @Override public String toString() { return "SealIndicesRequest{" + "indices=" + Arrays.toString(indices) + - ", indicesOptions=" + indicesOptions + + ", indicesOptions=" + indicesOptions() + '}'; } - - @Override - public ActionRequestValidationException validate() { - return null; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - indices = in.readStringArray(); - indicesOptions = IndicesOptions.readIndicesOptions(in); - } - - @Override - public SealIndicesRequest indices(String[] indices) { - this.indices = indices; - return this; - } - - public String[] indices() { - return indices; - } - - @Override - public IndicesOptions indicesOptions() { - return indicesOptions; - } - - @SuppressWarnings("unchecked") - public final SealIndicesRequest indicesOptions(IndicesOptions indicesOptions) { - this.indicesOptions = indicesOptions; - return this; - } } From 6ed472f873354be0726a8eb9708fc4a602543a40 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Sun, 17 May 2015 22:12:31 +0200 Subject: [PATCH 26/38] comments for SyncedFlushService --- .../indices/SyncedFlushService.java | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 5a5d4a42ebb..eafbc72f8c5 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -51,6 +51,7 @@ import org.elasticsearch.transport.*; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -110,7 +111,7 @@ public class SyncedFlushService extends AbstractComponent { * Tries to flush all copies of a shard and write a sync id to it. * After a synced flush two shard copies may only contain the same sync id if they contain the same documents. * To ensure this, synced flush works in three steps: - * 1. Flush all shard copies and gather the commit points for each copy after the flush + * 1. Flush all shard copies and gather the commit ids for each copy after the flush * 2. Ensure that there are no ongoing indexing operations on the primary * 3. Perform an additional flush on each shard copy that writes the sync id * @@ -143,7 +144,8 @@ public class SyncedFlushService extends AbstractComponent { } int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); - if (inflight != 1) { + assert inflight >= -1; + if (inflight != 1) { // 1 means that there are no write operations are in flight (>1) and the shard is not closed (0). actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); } String syncId = Strings.base64UUID(); @@ -213,6 +215,7 @@ public class SyncedFlushService extends AbstractComponent { } } catch (InterruptedException e) { logger.debug("{} interrupted while waiting for in flight operation check", shardId); + Thread.currentThread().interrupt(); } final int count = result.get(); logger.trace("{} in flight operation count [{}]", shardId, count); @@ -220,26 +223,22 @@ public class SyncedFlushService extends AbstractComponent { } - void sendSyncRequests(final String syncId, List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { - final CountDown countDownLatch = new CountDown(shards.size()); + void sendSyncRequests(final String syncId, final List shards, ClusterState state, Map expectedCommitIds, final ShardId shardId, final ActionListener listener) { + final CountDown countDown = new CountDown(shards.size()); final Map results = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); if (node == null) { logger.trace("{} is assigned to an unknown node. skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); results.put(shard, new SyncedFlushResponse("unknown node")); - if (countDownLatch.countDown()) { - listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); - } + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); continue; } final Engine.CommitId expectedCommitId = expectedCommitIds.get(shard.currentNodeId()); if (expectedCommitId == null) { logger.trace("{} can't resolve expected commit id for {}, skipping for sync id [{}]. shard routing {}", shardId, syncId, shard); results.put(shard, new SyncedFlushResponse("no commit id from pre-sync flush")); - if (countDownLatch.countDown()) { - listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); - } + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); continue; } logger.trace("{} sending synced flush request to {}. sync id [{}].", shardId, shard, syncId); @@ -255,18 +254,14 @@ public class SyncedFlushService extends AbstractComponent { SyncedFlushResponse existing = results.put(shard, response); assert existing == null : "got two answers for node [" + node + "]"; // count after the assert so we won't decrement twice in handleException - if (countDownLatch.countDown()) { - listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); - } + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); } @Override public void handleException(TransportException exp) { logger.trace("{} error while performing synced flush on [{}], skipping", exp, shardId, shard); results.put(shard, new SyncedFlushResponse(exp.getMessage())); - if (countDownLatch.countDown()) { - listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); - } + contDownAndSendResponseIfDone(syncId, shards, shardId, listener, countDown, results); } @Override @@ -278,12 +273,19 @@ public class SyncedFlushService extends AbstractComponent { } + private void contDownAndSendResponseIfDone(String syncId, List shards, ShardId shardId, ActionListener listener, CountDown countDown, Map results) { + if (countDown.countDown()) { + assert results.size() == shards.size(); + listener.onResponse(new SyncedFlushResult(shardId, syncId, results)); + } + } + /** * send presync requests to all started copies of the given shard */ Map sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId) { final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); - final Map commitIds = ConcurrentCollections.newConcurrentMap(); + final ConcurrentMap commitIds = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { logger.trace("{} sending pre-synced flush request to {}", shardId, shard); final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); @@ -300,7 +302,7 @@ public class SyncedFlushService extends AbstractComponent { @Override public void handleResponse(PreSyncedFlushResponse response) { - Engine.CommitId existing = commitIds.put(node.id(), response.commitId()); + Engine.CommitId existing = commitIds.putIfAbsent(node.id(), response.commitId()); assert existing == null : "got two answers for node [" + node + "]"; // count after the assert so we won't decrement twice in handleException countDownLatch.countDown(); @@ -324,6 +326,7 @@ public class SyncedFlushService extends AbstractComponent { } } catch (InterruptedException e) { logger.debug("{} interrupted while waiting for presync requests", shardId); + Thread.currentThread().interrupt(); } return commitIds; @@ -488,7 +491,6 @@ public class SyncedFlushService extends AbstractComponent { final static class PreSyncedFlushRequest extends TransportRequest { private ShardId shardId; - PreSyncedFlushRequest() { } From c4f9a41581474fb44d9d9f25fc7384d44db86576 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Mon, 18 May 2015 10:41:13 +0200 Subject: [PATCH 27/38] more comments and formatting --- .../java/org/elasticsearch/indices/SyncedFlushService.java | 3 +++ .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- src/test/java/org/elasticsearch/test/InternalTestCluster.java | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index eafbc72f8c5..6678b1c89dc 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -127,6 +127,9 @@ public class SyncedFlushService extends AbstractComponent { * changes on the replica (the write operation has not reached the replica yet). * * Step 2 detects this scenario and fails the whole synced flush if a write operation is ongoing on the primary. + * Together with the conditions for step 3 (same commit id and no uncommitted changes) this guarantees that a snc id will only + * be written on a primary if no write operation was executed between step 1 and step 3 and sync id will only be written on + * the replica if it contains the same changes that the primary contains. * * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies. * diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 93b6e5f030b..b9dd72aa72b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -415,7 +415,7 @@ public class RecoverySourceHandler { // related to this recovery (out of date segments, for example) // are deleted try { - final Store.MetadataSnapshot remainingFilesAfterCleanup = recoverWithSyncId? request.metadataSnapshot(): recoverySourceMetadata; + final Store.MetadataSnapshot remainingFilesAfterCleanup = recoverWithSyncId ? request.metadataSnapshot() : recoverySourceMetadata; transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), remainingFilesAfterCleanup, translogView.totalOperations()), TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), diff --git a/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/src/test/java/org/elasticsearch/test/InternalTestCluster.java index 505c9f49d48..c3436544b31 100644 --- a/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -977,6 +977,8 @@ public final class InternalTestCluster extends TestCluster { // and not all docs have been purged after the test) and inherit from // ElasticsearchIntegrationTest must override beforeIndexDeletion() to avoid failures. assertShardIndexCounter(); + //check that shards that have same sync id also contain same number of documents + } private void assertShardIndexCounter() { From f039c5de7ab9d413cadfa8d679f69a92e309111e Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Mon, 18 May 2015 11:06:36 +0200 Subject: [PATCH 28/38] add rest statuses but need to discuss if they are the right ones --- .../indices/seal/SealIndicesResponse.java | 48 +++++++++++++++++++ .../indices/seal/RestSealIndicesAction.java | 3 +- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java index 160f28efe63..1dfd47795a5 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/seal/SealIndicesResponse.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.indices.SyncedFlushService; +import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.util.*; @@ -37,12 +38,25 @@ public class SealIndicesResponse extends ActionResponse implements ToXContent { final private Set results; + private RestStatus restStatus; + SealIndicesResponse() { results = new HashSet<>(); } SealIndicesResponse(Set results) { this.results = results; + if (allShardsFailed()) { + restStatus = RestStatus.CONFLICT; + } else if (someShardsFailed()) { + restStatus = RestStatus.PARTIAL_CONTENT; + } else { + restStatus = RestStatus.OK; + } + } + + public RestStatus status() { + return restStatus; } @Override @@ -55,6 +69,7 @@ public class SealIndicesResponse extends ActionResponse implements ToXContent { syncedFlushResult.readFrom(in); results.add(syncedFlushResult); } + restStatus = RestStatus.readFrom(in); } @Override @@ -64,6 +79,7 @@ public class SealIndicesResponse extends ActionResponse implements ToXContent { for (SyncedFlushService.SyncedFlushResult syncedFlushResult : results) { syncedFlushResult.writeTo(out); } + RestStatus.writeTo(out, restStatus); } public Set results() { @@ -120,4 +136,36 @@ public class SealIndicesResponse extends ActionResponse implements ToXContent { } return builder; } + + public boolean allShardsFailed() { + for (SyncedFlushService.SyncedFlushResult result : results) { + if (result.success()) { + return false; + } + if (result.shardResponses().size() > 0) { + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + if (shardResponse.getValue().success()) { + return false; + } + } + } + } + return true; + } + + public boolean someShardsFailed() { + for (SyncedFlushService.SyncedFlushResult result : results) { + if (result.success() == false) { + return true; + } + if (result.shardResponses().size() > 0) { + for (Map.Entry shardResponse : result.shardResponses().entrySet()) { + if (shardResponse.getValue().success() == false) { + return true; + } + } + } + } + return false; + } } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java index e5953466f70..a2007ca703d 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/seal/RestSealIndicesAction.java @@ -33,7 +33,6 @@ import org.elasticsearch.rest.action.support.RestBuilderListener; import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.POST; -import static org.elasticsearch.rest.RestStatus.OK; /** * @@ -60,7 +59,7 @@ public class RestSealIndicesAction extends BaseRestHandler { builder.startObject(); builder = response.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); - return new BytesRestResponse(OK, builder); + return new BytesRestResponse(response.status(), builder); } }); } From 64956d2d87445b1142f983484c5c1d814fdfbc73 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Mon, 18 May 2015 11:27:49 +0200 Subject: [PATCH 29/38] add check at end of each test to see if documents are the same --- .../index/engine/CommitStats.java | 4 ++- .../test/InternalTestCluster.java | 28 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/index/engine/CommitStats.java b/src/main/java/org/elasticsearch/index/engine/CommitStats.java index de0474c5d7e..6e5d26c8e81 100644 --- a/src/main/java/org/elasticsearch/index/engine/CommitStats.java +++ b/src/main/java/org/elasticsearch/index/engine/CommitStats.java @@ -43,7 +43,9 @@ public final class CommitStats implements Streamable, ToXContent { userData = MapBuilder.newMapBuilder().putAll(segmentInfos.getUserData()).immutableMap(); // lucene calls the current generation, last generation. generation = segmentInfos.getLastGeneration(); - id = Base64.encodeBytes(segmentInfos.getId()); + if (segmentInfos.getId() != null) { // id is only written starting with Lucene 5.0 + id = Base64.encodeBytes(segmentInfos.getId()); + } } private CommitStats() { diff --git a/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/src/test/java/org/elasticsearch/test/InternalTestCluster.java index c3436544b31..24d317c2105 100644 --- a/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -74,6 +74,8 @@ import org.elasticsearch.index.cache.filter.FilterCacheModule; import org.elasticsearch.index.cache.filter.FilterCacheModule.FilterCacheSettings; import org.elasticsearch.index.cache.filter.index.IndexFilterCache; import org.elasticsearch.index.cache.filter.none.NoneFilterCache; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineClosedException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardModule; import org.elasticsearch.index.shard.IndexShardState; @@ -978,9 +980,35 @@ public final class InternalTestCluster extends TestCluster { // ElasticsearchIntegrationTest must override beforeIndexDeletion() to avoid failures. assertShardIndexCounter(); //check that shards that have same sync id also contain same number of documents + assertSameSyncIdSameDocs(); } + private void assertSameSyncIdSameDocs() { + Map docsOnShards = new HashMap<>(); + final Collection nodesAndClients = nodes.values(); + for (NodeAndClient nodeAndClient : nodesAndClients) { + IndicesService indexServices = getInstance(IndicesService.class, nodeAndClient.name); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + try { + String syncId = indexShard.engine().commitStats().getUserData().get(Engine.SYNC_COMMIT_ID); + if (syncId != null) { + long liveDocsOnShard = indexShard.docStats().getCount() - indexShard.docStats().getDeleted(); + if (docsOnShards.get(syncId) != null) { + assertThat("sync id is equal but number of docs does not match on node " + nodeAndClient.name + ". expected " + docsOnShards.get(syncId) + " but got " + liveDocsOnShard, docsOnShards.get(syncId), equalTo(liveDocsOnShard)); + } else { + docsOnShards.put(syncId, liveDocsOnShard); + } + } + } catch (EngineClosedException e) { + // nothing to do, shard is closed + } + } + } + } + } + private void assertShardIndexCounter() { final Collection nodesAndClients = nodes.values(); for (NodeAndClient nodeAndClient : nodesAndClients) { From 6dbe7d4b55b2ff0aef04c94676d4af6e670ff1fd Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 18 May 2015 21:55:23 +0200 Subject: [PATCH 30/38] Make commitID byte[] final --- src/main/java/org/elasticsearch/index/engine/Engine.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 1fa12398f21..b44f244140a 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -1147,7 +1147,9 @@ public abstract class Engine implements Closeable { public abstract boolean hasUncommittedChanges(); public static class CommitId implements Writeable { - private byte[] id; + + private final byte[] id; + public CommitId(byte[] id) { assert id != null; this.id = Arrays.copyOf(id, id.length); @@ -1165,8 +1167,7 @@ public abstract class Engine implements Closeable { @Override public CommitId readFrom(StreamInput in) throws IOException { - byte[] bytes = in.readByteArray(); - return new CommitId(bytes); + return new CommitId(in); } @Override From 8ef734908ce9112e18ef1bc7d1e780efe4f51fb3 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Tue, 19 May 2015 09:28:04 +0200 Subject: [PATCH 31/38] assert num docs in recovery if sync id was used --- .../elasticsearch/common/lucene/Lucene.java | 11 ++++++ .../BlobStoreIndexShardRepository.java | 2 +- .../org/elasticsearch/index/store/Store.java | 39 +++++++++++++++---- .../recovery/RecoverySourceHandler.java | 1 + .../elasticsearch/index/store/StoreTest.java | 9 ++--- 5 files changed, 49 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 92593d13479..8e1c4aa2fe3 100644 --- a/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -141,6 +141,17 @@ public class Lucene { return Iterables.concat(list.toArray(new Collection[0])); } + /** + * Returns the number of document in the index referenced by this {@link SegmentInfos} + */ + public static int getNumDocs(SegmentInfos info) { + int numDocs = 0; + for (SegmentCommitInfo si : info) { + numDocs += si.info.maxDoc() - si.getDelCount(); + } + return numDocs; + } + /** * Reads the segments infos from the given commit, failing if it fails to load */ diff --git a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java index 7e9360ccbbc..707e77dbefe 100644 --- a/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java +++ b/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardRepository.java @@ -734,7 +734,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); fileInfos.put(fileInfo.metadata().name(), fileInfo); } - final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData, Collections.EMPTY_MAP); + final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(snapshotMetaData, Collections.EMPTY_MAP, 0); final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); for (StoreFileMetaData md : diff.identical) { FileInfo fileInfo = fileInfos.get(md.name()); diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index cf6557002b1..2db9bcafd8a 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; @@ -669,26 +668,32 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref private final ImmutableMap commitUserData; - public MetadataSnapshot(Map metadata, Map commitUserData) { + private final long numDocs; + + public MetadataSnapshot(Map metadata, Map commitUserData, long numDocs) { ImmutableMap.Builder metaDataBuilder = ImmutableMap.builder(); this.metadata = metaDataBuilder.putAll(metadata).build(); ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); this.commitUserData = commitUserDataBuilder.putAll(commitUserData).build(); + this.numDocs = numDocs; } MetadataSnapshot() { metadata = ImmutableMap.of(); commitUserData = ImmutableMap.of(); + numDocs = 0; } MetadataSnapshot(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { - Tuple, ImmutableMap> loadedMetadata = loadMetadata(commit, directory, logger); - metadata = loadedMetadata.v1(); - commitUserData = loadedMetadata.v2(); + LoadedMetadata loadedMetadata = loadMetadata(commit, directory, logger); + metadata = loadedMetadata.fileMetadata; + commitUserData = loadedMetadata.userData; + numDocs = loadedMetadata.numDocs; assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); } public MetadataSnapshot(StreamInput in) throws IOException { + int size = in.readVInt(); ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); for (int i = 0; i < size; i++) { @@ -703,15 +708,34 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref this.commitUserData = commitUserDataBuilder.build(); this.metadata = metadataBuilder.build(); + this.numDocs = in.readLong(); assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); } - static Tuple, ImmutableMap> loadMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { + public long getNumDocs() { + return numDocs; + } + + public static class LoadedMetadata { + ImmutableMap fileMetadata; + ImmutableMap userData; + long numDocs; + + public LoadedMetadata(ImmutableMap fileMetadata, ImmutableMap userData, long numDocs) { + this.fileMetadata = fileMetadata; + this.userData = userData; + this.numDocs = numDocs; + } + } + + static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, ESLogger logger) throws IOException { + long numDocs; ImmutableMap.Builder builder = ImmutableMap.builder(); Map checksumMap = readLegacyChecksums(directory).v1(); ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); try { final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory); + numDocs = Lucene.getNumDocs(segmentCommitInfos); commitUserDataBuilder.putAll(segmentCommitInfos.getUserData()); Version maxVersion = Version.LUCENE_4_0; // we don't know which version was used to write so we take the max version. for (SegmentCommitInfo info : segmentCommitInfos) { @@ -765,7 +789,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref throw ex; } - return new Tuple, ImmutableMap>(builder.build(), commitUserDataBuilder.build()); + return new LoadedMetadata(builder.build(), commitUserDataBuilder.build(), numDocs); } /** @@ -997,6 +1021,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref out.writeString(entry.getKey()); out.writeString(entry.getValue()); } + out.writeLong(numDocs); } public Map getCommitUserData() { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index b9dd72aa72b..ee36f9bcb7e 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -207,6 +207,7 @@ public class RecoverySourceHandler { final boolean recoverWithSyncId = recoverySourceSyncId != null && recoverySourceSyncId.equals(recoveryTargetSyncId); if (recoverWithSyncId) { + assert request.metadataSnapshot().getNumDocs() == recoverySourceMetadata.getNumDocs(); for (StoreFileMetaData md : request.metadataSnapshot()) { response.phase1ExistingFileNames.add(md.name()); response.phase1ExistingFileSizes.add(md.length()); diff --git a/src/test/java/org/elasticsearch/index/store/StoreTest.java b/src/test/java/org/elasticsearch/index/store/StoreTest.java index ee244c388b8..0fa44157646 100644 --- a/src/test/java/org/elasticsearch/index/store/StoreTest.java +++ b/src/test/java/org/elasticsearch/index/store/StoreTest.java @@ -54,7 +54,6 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; @@ -728,9 +727,9 @@ public class StoreTest extends ElasticsearchTestCase { Map metaDataMap = new HashMap<>(); metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); + Store.MetadataSnapshot first = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); - Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); + Store.MetadataSnapshot second = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); Store.RecoveryDiff recoveryDiff = first.recoveryDiff(second); assertEquals(recoveryDiff.toString(), recoveryDiff.different.size(), 2); } @@ -1000,7 +999,7 @@ public class StoreTest extends ElasticsearchTestCase { Map metaDataMap = new HashMap<>(); metaDataMap.put("segments_1", new StoreFileMetaData("segments_1", 50, null, null, new BytesRef(new byte[]{1}))); metaDataMap.put("_0_1.del", new StoreFileMetaData("_0_1.del", 42, "foobarbaz", null, new BytesRef())); - Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP); + Store.MetadataSnapshot snapshot = new Store.MetadataSnapshot(metaDataMap, Collections.EMPTY_MAP, 0); final ShardId shardId = new ShardId(new Index("index"), 1); DirectoryService directoryService = new LuceneManagedDirectoryService(random()); @@ -1134,7 +1133,7 @@ public class StoreTest extends ElasticsearchTestCase { Map commitUserData = new HashMap<>(); commitUserData.put("userdata_1", "test"); commitUserData.put("userdata_2", "test"); - return new Store.MetadataSnapshot(storeFileMetaDataMap, commitUserData); + return new Store.MetadataSnapshot(storeFileMetaDataMap, commitUserData, 0); } @Test From 8a3d504efb1a09130ece7f9c692f0a51f51beb74 Mon Sep 17 00:00:00 2001 From: Britta Weber Date: Mon, 18 May 2015 22:30:57 +0200 Subject: [PATCH 32/38] skip everything between file checks and translog prepare if sync ids coincide also throw exception instead of assert if num docs no coincide --- .../recovery/RecoverySourceHandler.java | 423 +++++++++--------- 1 file changed, 211 insertions(+), 212 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index ee36f9bcb7e..a1426ea120b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -50,6 +50,7 @@ import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.CancellableThreads.Interruptable; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.engine.Engine; @@ -131,7 +132,9 @@ public class RecoverySourceHandler { this.response = new RecoveryResponse(); } - /** performs the recovery from the local engine to the target */ + /** + * performs the recovery from the local engine to the target + */ public RecoveryResponse recoverToTarget() { final Engine engine = shard.engine(); assert engine.getTranslog() != null : "translog must not be null"; @@ -207,17 +210,13 @@ public class RecoverySourceHandler { final boolean recoverWithSyncId = recoverySourceSyncId != null && recoverySourceSyncId.equals(recoveryTargetSyncId); if (recoverWithSyncId) { - assert request.metadataSnapshot().getNumDocs() == recoverySourceMetadata.getNumDocs(); - for (StoreFileMetaData md : request.metadataSnapshot()) { - response.phase1ExistingFileNames.add(md.name()); - response.phase1ExistingFileSizes.add(md.length()); - existingTotalSize += md.length(); - if (logger.isTraceEnabled()) { - logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], checksum [{}], size [{}], sync ids {} coincide, will skip file copy", - indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length(), recoverySourceMetadata.getCommitUserData().get(Engine.SYNC_COMMIT_ID)); - } - totalSize += md.length(); + long numDocsTarget = request.metadataSnapshot().getNumDocs(); + long numDocsSource = recoverySourceMetadata.getNumDocs(); + if (numDocsTarget != numDocsSource) { + throw new IllegalStateException("try to recover " + request.shardId() + " from primary shard with sync id but number of docs differ: " + numDocsTarget + " (" + request.sourceNode().getName() + ", primary) vs " + numDocsSource + "(" + request.targetNode().getName() + ")"); } + // we shortcut recovery here because we have nothing to copy. but we must still start the engine on the target. + // so we don't return here } else { final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(request.metadataSnapshot()); for (StoreFileMetaData md : diff.identical) { @@ -242,224 +241,224 @@ public class RecoverySourceHandler { response.phase1FileSizes.add(md.length()); totalSize += md.length(); } - } - response.phase1TotalSize = totalSize; - response.phase1ExistingTotalSize = existingTotalSize; - logger.trace("[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", - indexName, shardId, request.targetNode(), response.phase1FileNames.size(), - new ByteSizeValue(totalSize), response.phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), - response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes, - translogView.totalOperations()); - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, - TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + response.phase1TotalSize = totalSize; + response.phase1ExistingTotalSize = existingTotalSize; + + logger.trace("[{}][{}] recovery [phase1] to {}: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", + indexName, shardId, request.targetNode(), response.phase1FileNames.size(), + new ByteSizeValue(totalSize), response.phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); + cancellableThreads.execute(new Interruptable() { + @Override + public void run() throws InterruptedException { + RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), + response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes, + translogView.totalOperations()); + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, + TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } + }); + + // This latch will be used to wait until all files have been transferred to the target node + final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size()); + final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); + final AtomicReference corruptedEngine = new AtomicReference<>(); + int fileIndex = 0; + ThreadPoolExecutor pool; + + // How many bytes we've copied since we last called RateLimiter.pause + final AtomicLong bytesSinceLastPause = new AtomicLong(); + + for (final String name : response.phase1FileNames) { + long fileSize = response.phase1FileSizes.get(fileIndex); + + // Files are split into two categories, files that are "small" + // (under 5mb) and other files. Small files are transferred + // using a separate thread pool dedicated to small files. + // + // The idea behind this is that while we are transferring an + // older, large index, a user may create a new index, but that + // index will not be able to recover until the large index + // finishes, by using two different thread pools we can allow + // tiny files (like segments for a brand new index) to be + // recovered while ongoing large segment recoveries are + // happening. It also allows these pools to be configured + // separately. + if (fileSize > RecoverySettings.SMALL_FILE_CUTOFF_BYTES) { + pool = recoverySettings.concurrentStreamPool(); + } else { + pool = recoverySettings.concurrentSmallFileStreamPool(); + } + + pool.execute(new AbstractRunnable() { + @Override + public void onFailure(Throwable t) { + // we either got rejected or the store can't be incremented / we are canceled + logger.debug("Failed to transfer file [" + name + "] on recovery"); + } + + @Override + public void onAfter() { + // Signify this file has completed by decrementing the latch + latch.countDown(); + } + + @Override + protected void doRun() { + cancellableThreads.checkForCancel(); + store.incRef(); + final StoreFileMetaData md = recoverySourceMetadata.get(name); + try (final IndexInput indexInput = store.directory().openInput(name, IOContext.READONCE)) { + final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes(); + final byte[] buf = new byte[BUFFER_SIZE]; + boolean shouldCompressRequest = recoverySettings.compress(); + if (CompressorFactory.isCompressed(indexInput)) { + shouldCompressRequest = false; + } + + final long len = indexInput.length(); + long readCount = 0; + final TransportRequestOptions requestOptions = TransportRequestOptions.options() + .withCompress(shouldCompressRequest) + .withType(TransportRequestOptions.Type.RECOVERY) + .withTimeout(recoverySettings.internalActionTimeout()); + + while (readCount < len) { + if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us + throw new IndexShardClosedException(shard.shardId()); + } + int toRead = readCount + BUFFER_SIZE > len ? (int) (len - readCount) : BUFFER_SIZE; + final long position = indexInput.getFilePointer(); + + // Pause using the rate limiter, if desired, to throttle the recovery + RateLimiter rl = recoverySettings.rateLimiter(); + long throttleTimeInNanos = 0; + if (rl != null) { + long bytes = bytesSinceLastPause.addAndGet(toRead); + if (bytes > rl.getMinPauseCheckBytes()) { + // Time to pause + bytesSinceLastPause.addAndGet(-bytes); + throttleTimeInNanos = rl.pause(bytes); + shard.recoveryStats().addThrottleTime(throttleTimeInNanos); + } + } + indexInput.readBytes(buf, 0, toRead, false); + final BytesArray content = new BytesArray(buf, 0, toRead); + readCount += toRead; + final boolean lastChunk = readCount == len; + final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, + content, lastChunk, translogView.totalOperations(), throttleTimeInNanos); + cancellableThreads.execute(new Interruptable() { + @Override + public void run() throws InterruptedException { + // Actually send the file chunk to the target node, waiting for it to complete + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, + fileChunkRequest, requestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } + }); + + } + } catch (Throwable e) { + final Throwable corruptIndexException; + if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); + if (corruptedEngine.compareAndSet(null, corruptIndexException) == false) { + // if we are not the first exception, add ourselves as suppressed to the main one: + corruptedEngine.get().addSuppressed(e); + } + } else { // corruption has happened on the way to replica + RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); + exception.addSuppressed(e); + exceptions.add(0, exception); // last exception first + logger.warn("{} Remote file corruption on node {}, recovering {}. local checksum OK", + corruptIndexException, shard.shardId(), request.targetNode(), md); + + } + } else { + exceptions.add(0, e); // last exceptions first + } + } finally { + store.decRef(); + + } + } + }); + fileIndex++; } - }); - // This latch will be used to wait until all files have been transferred to the target node - final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size()); - final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); - final AtomicReference corruptedEngine = new AtomicReference<>(); - int fileIndex = 0; - ThreadPoolExecutor pool; + cancellableThreads.execute(new Interruptable() { + @Override + public void run() throws InterruptedException { + // Wait for all files that need to be transferred to finish transferring + latch.await(); + } + }); - // How many bytes we've copied since we last called RateLimiter.pause - final AtomicLong bytesSinceLastPause = new AtomicLong(); - - for (final String name : response.phase1FileNames) { - long fileSize = response.phase1FileSizes.get(fileIndex); - - // Files are split into two categories, files that are "small" - // (under 5mb) and other files. Small files are transferred - // using a separate thread pool dedicated to small files. - // - // The idea behind this is that while we are transferring an - // older, large index, a user may create a new index, but that - // index will not be able to recover until the large index - // finishes, by using two different thread pools we can allow - // tiny files (like segments for a brand new index) to be - // recovered while ongoing large segment recoveries are - // happening. It also allows these pools to be configured - // separately. - if (fileSize > RecoverySettings.SMALL_FILE_CUTOFF_BYTES) { - pool = recoverySettings.concurrentStreamPool(); + if (corruptedEngine.get() != null) { + shard.engine().failEngine("recovery", corruptedEngine.get()); + throw corruptedEngine.get(); } else { - pool = recoverySettings.concurrentSmallFileStreamPool(); + ExceptionsHelper.rethrowAndSuppress(exceptions); } - pool.execute(new AbstractRunnable() { + cancellableThreads.execute(new Interruptable() { @Override - public void onFailure(Throwable t) { - // we either got rejected or the store can't be incremented / we are canceled - logger.debug("Failed to transfer file [" + name + "] on recovery"); - } - - @Override - public void onAfter() { - // Signify this file has completed by decrementing the latch - latch.countDown(); - } - - @Override - protected void doRun() { - cancellableThreads.checkForCancel(); - store.incRef(); - final StoreFileMetaData md = recoverySourceMetadata.get(name); - try (final IndexInput indexInput = store.directory().openInput(name, IOContext.READONCE)) { - final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes(); - final byte[] buf = new byte[BUFFER_SIZE]; - boolean shouldCompressRequest = recoverySettings.compress(); - if (CompressorFactory.isCompressed(indexInput)) { - shouldCompressRequest = false; - } - - final long len = indexInput.length(); - long readCount = 0; - final TransportRequestOptions requestOptions = TransportRequestOptions.options() - .withCompress(shouldCompressRequest) - .withType(TransportRequestOptions.Type.RECOVERY) - .withTimeout(recoverySettings.internalActionTimeout()); - - while (readCount < len) { - if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us - throw new IndexShardClosedException(shard.shardId()); - } - int toRead = readCount + BUFFER_SIZE > len ? (int) (len - readCount) : BUFFER_SIZE; - final long position = indexInput.getFilePointer(); - - // Pause using the rate limiter, if desired, to throttle the recovery - RateLimiter rl = recoverySettings.rateLimiter(); - long throttleTimeInNanos = 0; - if (rl != null) { - long bytes = bytesSinceLastPause.addAndGet(toRead); - if (bytes > rl.getMinPauseCheckBytes()) { - // Time to pause - bytesSinceLastPause.addAndGet(-bytes); - throttleTimeInNanos = rl.pause(bytes); - shard.recoveryStats().addThrottleTime(throttleTimeInNanos); + public void run() throws InterruptedException { + // Send the CLEAN_FILES request, which takes all of the files that + // were transferred and renames them from their temporary file + // names to the actual file names. It also writes checksums for + // the files after they have been renamed. + // + // Once the files have been renamed, any other files that are not + // related to this recovery (out of date segments, for example) + // are deleted + try { + transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, + new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, translogView.totalOperations()), + TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), + EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + } catch (RemoteTransportException remoteException) { + final IOException corruptIndexException; + // we realized that after the index was copied and we wanted to finalize the recovery + // the index was corrupted: + // - maybe due to a broken segments file on an empty index (transferred with no checksum) + // - maybe due to old segments without checksums or length only checks + if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(remoteException)) != null) { + try { + final Store.MetadataSnapshot recoverySourceMetadata = store.getMetadata(snapshot); + StoreFileMetaData[] metadata = Iterables.toArray(recoverySourceMetadata, StoreFileMetaData.class); + ArrayUtil.timSort(metadata, new Comparator() { + @Override + public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { + return Long.compare(o1.length(), o2.length()); // check small files first + } + }); + for (StoreFileMetaData md : metadata) { + logger.debug("{} checking integrity for file {} after remove corruption exception", shard.shardId(), md); + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); + throw corruptIndexException; + } } + } catch (IOException ex) { + remoteException.addSuppressed(ex); + throw remoteException; } - indexInput.readBytes(buf, 0, toRead, false); - final BytesArray content = new BytesArray(buf, 0, toRead); - readCount += toRead; - final boolean lastChunk = readCount == len; - final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, - content, lastChunk, translogView.totalOperations(), throttleTimeInNanos); - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Actually send the file chunk to the target node, waiting for it to complete - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, - fileChunkRequest, requestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - } - }); - - } - } catch (Throwable e) { - final Throwable corruptIndexException; - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); - if (corruptedEngine.compareAndSet(null, corruptIndexException) == false) { - // if we are not the first exception, add ourselves as suppressed to the main one: - corruptedEngine.get().addSuppressed(e); - } - } else { // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); - exception.addSuppressed(e); - exceptions.add(0, exception); // last exception first - logger.warn("{} Remote file corruption on node {}, recovering {}. local checksum OK", - corruptIndexException, shard.shardId(), request.targetNode(), md); - - } + // corruption has happened on the way to replica + RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); + exception.addSuppressed(remoteException); + logger.warn("{} Remote file corruption during finalization on node {}, recovering {}. local checksum OK", + corruptIndexException, shard.shardId(), request.targetNode()); } else { - exceptions.add(0, e); // last exceptions first + throw remoteException; } - } finally { - store.decRef(); - } } }); - fileIndex++; } - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Wait for all files that need to be transferred to finish transferring - latch.await(); - } - }); - - if (corruptedEngine.get() != null) { - shard.engine().failEngine("recovery", corruptedEngine.get()); - throw corruptedEngine.get(); - } else { - ExceptionsHelper.rethrowAndSuppress(exceptions); - } - - cancellableThreads.execute(new Interruptable() { - @Override - public void run() throws InterruptedException { - // Send the CLEAN_FILES request, which takes all of the files that - // were transferred and renames them from their temporary file - // names to the actual file names. It also writes checksums for - // the files after they have been renamed. - // - // Once the files have been renamed, any other files that are not - // related to this recovery (out of date segments, for example) - // are deleted - try { - final Store.MetadataSnapshot remainingFilesAfterCleanup = recoverWithSyncId ? request.metadataSnapshot() : recoverySourceMetadata; - transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, - new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), remainingFilesAfterCleanup, translogView.totalOperations()), - TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); - } catch (RemoteTransportException remoteException) { - final IOException corruptIndexException; - // we realized that after the index was copied and we wanted to finalize the recovery - // the index was corrupted: - // - maybe due to a broken segments file on an empty index (transferred with no checksum) - // - maybe due to old segments without checksums or length only checks - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(remoteException)) != null) { - try { - final Store.MetadataSnapshot recoverySourceMetadata = store.getMetadata(snapshot); - StoreFileMetaData[] metadata = Iterables.toArray(recoverySourceMetadata, StoreFileMetaData.class); - ArrayUtil.timSort(metadata, new Comparator() { - @Override - public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { - return Long.compare(o1.length(), o2.length()); // check small files first - } - }); - for (StoreFileMetaData md : metadata) { - logger.debug("{} checking integrity for file {} after remove corruption exception", shard.shardId(), md); - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md); - throw corruptIndexException; - } - } - } catch (IOException ex) { - remoteException.addSuppressed(ex); - throw remoteException; - } - // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but checksums are ok", null); - exception.addSuppressed(remoteException); - logger.warn("{} Remote file corruption during finalization on node {}, recovering {}. local checksum OK", - corruptIndexException, shard.shardId(), request.targetNode()); - } else { - throw remoteException; - } - } - } - }); - prepareTargetForTranslog(translogView); logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime()); From 09283cf7d3ed234fecb76724d6dec366e3e2461e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 10:23:18 +0200 Subject: [PATCH 33/38] minor cleanups --- .../org/elasticsearch/index/store/Store.java | 20 ++++++++++--------- .../recovery/RecoverySourceHandler.java | 4 ++-- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index 2db9bcafd8a..edcd930b8d6 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -693,14 +693,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref } public MetadataSnapshot(StreamInput in) throws IOException { - - int size = in.readVInt(); - ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); + final int size = in.readVInt(); + final ImmutableMap.Builder metadataBuilder = ImmutableMap.builder(); for (int i = 0; i < size; i++) { StoreFileMetaData meta = StoreFileMetaData.readStoreFileMetaData(in); metadataBuilder.put(meta.name(), meta); } - ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); + final ImmutableMap.Builder commitUserDataBuilder = ImmutableMap.builder(); int num = in.readVInt(); for (int i = num; i > 0; i--) { commitUserDataBuilder.put(in.readString(), in.readString()); @@ -712,16 +711,19 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref assert metadata.isEmpty() || numSegmentFiles() == 1 : "numSegmentFiles: " + numSegmentFiles(); } + /** + * Returns the number of documents in this store snapshot + */ public long getNumDocs() { return numDocs; } - public static class LoadedMetadata { - ImmutableMap fileMetadata; - ImmutableMap userData; - long numDocs; + static class LoadedMetadata { + final ImmutableMap fileMetadata; + final ImmutableMap userData; + final long numDocs; - public LoadedMetadata(ImmutableMap fileMetadata, ImmutableMap userData, long numDocs) { + LoadedMetadata(ImmutableMap fileMetadata, ImmutableMap userData, long numDocs) { this.fileMetadata = fileMetadata; this.userData = userData; this.numDocs = numDocs; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index a1426ea120b..764a8f016d3 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -210,8 +210,8 @@ public class RecoverySourceHandler { final boolean recoverWithSyncId = recoverySourceSyncId != null && recoverySourceSyncId.equals(recoveryTargetSyncId); if (recoverWithSyncId) { - long numDocsTarget = request.metadataSnapshot().getNumDocs(); - long numDocsSource = recoverySourceMetadata.getNumDocs(); + final long numDocsTarget = request.metadataSnapshot().getNumDocs(); + final long numDocsSource = recoverySourceMetadata.getNumDocs(); if (numDocsTarget != numDocsSource) { throw new IllegalStateException("try to recover " + request.shardId() + " from primary shard with sync id but number of docs differ: " + numDocsTarget + " (" + request.sourceNode().getName() + ", primary) vs " + numDocsSource + "(" + request.targetNode().getName() + ")"); } From d817793f9b86e5ef8a848e2f6373f792558a022b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 10:32:40 +0200 Subject: [PATCH 34/38] [TEST] Add Unittest for Lucene#getNumDocs --- .../common/lucene/LuceneTest.java | 52 ++++++++++++++++++- 1 file changed, 50 insertions(+), 2 deletions(-) diff --git a/src/test/java/org/elasticsearch/common/lucene/LuceneTest.java b/src/test/java/org/elasticsearch/common/lucene/LuceneTest.java index 18d77305a40..85852cf9f91 100644 --- a/src/test/java/org/elasticsearch/common/lucene/LuceneTest.java +++ b/src/test/java/org/elasticsearch/common/lucene/LuceneTest.java @@ -30,8 +30,7 @@ import org.elasticsearch.test.ElasticsearchTestCase; import org.junit.Test; import java.io.IOException; -import java.util.HashSet; -import java.util.Set; +import java.util.*; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -272,6 +271,55 @@ public class LuceneTest extends ElasticsearchTestCase { assertTrue(files.toString(), files.contains("_1.si")); writer.close(); dir.close(); + } + public void testNumDocs() throws IOException { + MockDirectoryWrapper dir = newMockDirectory(); + dir.setEnableVirusScanner(false); + IndexWriterConfig iwc = newIndexWriterConfig(); + IndexWriter writer = new IndexWriter(dir, iwc); + Document doc = new Document(); + doc.add(new TextField("id", "1", random().nextBoolean() ? Field.Store.YES : Field.Store.NO)); + writer.addDocument(doc); + writer.commit(); + SegmentInfos segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(1, Lucene.getNumDocs(segmentCommitInfos)); + + doc = new Document(); + doc.add(new TextField("id", "2", random().nextBoolean() ? Field.Store.YES : Field.Store.NO)); + writer.addDocument(doc); + + doc = new Document(); + doc.add(new TextField("id", "3", random().nextBoolean() ? Field.Store.YES : Field.Store.NO)); + writer.addDocument(doc); + segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(1, Lucene.getNumDocs(segmentCommitInfos)); + writer.commit(); + segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(3, Lucene.getNumDocs(segmentCommitInfos)); + writer.deleteDocuments(new Term("id", "2")); + writer.commit(); + segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(2, Lucene.getNumDocs(segmentCommitInfos)); + + int numDocsToIndex = randomIntBetween(10, 50); + List deleteTerms = new ArrayList<>(); + for (int i = 0; i < numDocsToIndex; i++) { + doc = new Document(); + doc.add(new TextField("id", "extra_" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO)); + deleteTerms.add(new Term("id", "extra_" + i)); + writer.addDocument(doc); + } + int numDocsToDelete = randomIntBetween(0, numDocsToIndex); + Collections.shuffle(deleteTerms, random()); + for (int i = 0; i < numDocsToDelete; i++) { + Term remove = deleteTerms.remove(0); + writer.deleteDocuments(remove); + } + writer.commit(); + segmentCommitInfos = Lucene.readSegmentInfos(dir); + assertEquals(2 + deleteTerms.size(), Lucene.getNumDocs(segmentCommitInfos)); + writer.close(); + dir.close(); } } From 55f99dfce6e763e0ceb0259b45933de26a8b7436 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 12:01:31 +0200 Subject: [PATCH 35/38] Make SyncedFlushService fully asynchronous Some requests in the SyncedFlushService were sill blocking on network calls which made calling this service error prone if done on a network thread. This commit makes this service fully async based on ActionListener. --- .../indices/SyncedFlushService.java | 113 +++++++++--------- .../gateway/RecoveryFromGatewayTests.java | 3 +- .../org/elasticsearch/indices/FlushTest.java | 1 - .../indices/SycnedFlushSingleNodeTest.java | 27 ++--- .../{test => indices}/SyncedFlushUtil.java | 36 ++++-- 5 files changed, 97 insertions(+), 83 deletions(-) rename src/test/java/org/elasticsearch/{test => indices}/SyncedFlushUtil.java (60%) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index 6678b1c89dc..fc0a1b50df1 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexShardMissingException; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardException; import org.elasticsearch.index.shard.ShardId; @@ -62,16 +63,9 @@ public class SyncedFlushService extends AbstractComponent { public static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; public static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; - public static final String SETTING_PRE_SYNC_TIMEOUT = "indices.flush.synced.presync_timeout"; - public static final String SETTING_SYNC_TIMEOUT = "indices.flush.synced.sync_timeout"; - public static final String SETTING_IN_FLIGHT_OPS_TIMEOUT = "indices.flush.synced.in_flight_ops_timeout"; - private final IndicesService indicesService; private final ClusterService clusterService; private final TransportService transportService; - private final TimeValue preSyncTimeout; - private final TimeValue syncTimeout; - private final TimeValue inflightOpsTimeout; @Inject public SyncedFlushService(Settings settings, IndicesService indicesService, ClusterService clusterService, TransportService transportService) { @@ -83,9 +77,6 @@ public class SyncedFlushService extends AbstractComponent { transportService.registerRequestHandler(PRE_SYNCED_FLUSH_ACTION_NAME, PreSyncedFlushRequest.class, ThreadPool.Names.FLUSH, new PreSyncedFlushTransportHandler()); transportService.registerRequestHandler(SYNCED_FLUSH_ACTION_NAME, SyncedFlushRequest.class, ThreadPool.Names.FLUSH, new SyncedFlushTransportHandler()); transportService.registerRequestHandler(IN_FLIGHT_OPS_ACTION_NAME, InFlightOpsRequest.class, ThreadPool.Names.SAME, new InFlightOpCountTransportHandler()); - preSyncTimeout = settings.getAsTime(SETTING_PRE_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); - syncTimeout = settings.getAsTime(SETTING_SYNC_TIMEOUT, TimeValue.timeValueMinutes(5)); - inflightOpsTimeout = settings.getAsTime(SETTING_IN_FLIGHT_OPS_TIMEOUT, TimeValue.timeValueMinutes(5)); indicesService.indicesLifecycle().addListener(new IndicesLifecycle.Listener() { @Override public void onShardInactive(final IndexShard indexShard) { @@ -132,27 +123,49 @@ public class SyncedFlushService extends AbstractComponent { * the replica if it contains the same changes that the primary contains. * * Synced flush is a best effort operation. The sync id may be written on all, some or none of the copies. - * - * **/ - - public void attemptSyncedFlush(ShardId shardId, ActionListener actionListener) { + **/ + public void attemptSyncedFlush(final ShardId shardId, final ActionListener actionListener) { try { final ClusterState state = clusterService.state(); final IndexShardRoutingTable shardRoutingTable = getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); - Map commitIds = sendPreSyncRequests(activeShards, state, shardId); + final ActionListener> commitIdsListener = new ActionListener>() { + @Override + public void onResponse(final Map commitIds) { + if (commitIds.isEmpty()) { + actionListener.onResponse(new SyncedFlushResult(shardId, "all shards failed to commit on pre-sync")); + } + final ActionListener inflightOpsListener = new ActionListener() { + @Override + public void onResponse(InFlightOpsResponse response) { + final int inflight = response.opCount(); + assert inflight >= -1; + if (inflight != 1) { // 1 means that there are no write operations are in flight (>1) and the shard is not closed (0). + actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); + } else { + // 3. now send the sync request to all the shards + String syncId = Strings.base64UUID(); + sendSyncRequests(syncId, activeShards, state, commitIds, shardId, actionListener); + } + } - if (commitIds.isEmpty()) { - actionListener.onResponse(new SyncedFlushResult(shardId, "all shards failed to commit on pre-sync")); - } + @Override + public void onFailure(Throwable e) { + actionListener.onFailure(e); + } + }; + // 2. fetch in flight operations + getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsListener); + } - int inflight = getInflightOpsCount(shardId, state, shardRoutingTable); - assert inflight >= -1; - if (inflight != 1) { // 1 means that there are no write operations are in flight (>1) and the shard is not closed (0). - actionListener.onResponse(new SyncedFlushResult(shardId, "operation counter on primary is non zero [" + inflight + "]")); - } - String syncId = Strings.base64UUID(); - sendSyncRequests(syncId, activeShards, state, commitIds, shardId, actionListener); + @Override + public void onFailure(Throwable e) { + actionListener.onFailure(e); + } + }; + + // 1. send pre-sync flushes to all replicas + sendPreSyncRequests(activeShards, state, shardId, commitIdsListener); } catch (Throwable t) { actionListener.onFailure(t); } @@ -177,15 +190,14 @@ public class SyncedFlushService extends AbstractComponent { /** * returns the number of inflight operations on primary. -1 upon error. */ - protected int getInflightOpsCount(final ShardId shardId, ClusterState state, IndexShardRoutingTable shardRoutingTable) { + protected void getInflightOpsCount(final ShardId shardId, ClusterState state, IndexShardRoutingTable shardRoutingTable, final ActionListener listener) { final ShardRouting primaryShard = shardRoutingTable.primaryShard(); final DiscoveryNode primaryNode = state.nodes().get(primaryShard.currentNodeId()); if (primaryNode == null) { logger.trace("{} failed to resolve node for primary shard {}, skipping sync", shardId, primaryShard); - return -1; + listener.onResponse(new InFlightOpsResponse(-1)); + return; } - final AtomicInteger result = new AtomicInteger(-1); - final CountDownLatch latch = new CountDownLatch(1); logger.trace("{} retrieving in flight operation count", shardId); transportService.sendRequest(primaryNode, IN_FLIGHT_OPS_ACTION_NAME, new InFlightOpsRequest(shardId), new BaseTransportResponseHandler() { @@ -196,15 +208,13 @@ public class SyncedFlushService extends AbstractComponent { @Override public void handleResponse(InFlightOpsResponse response) { - result.set(response.opCount()); - latch.countDown(); + listener.onResponse(response); } @Override public void handleException(TransportException exp) { logger.debug("{} unexpected error while retrieving inflight op count", shardId); - result.set(-1); - latch.countDown(); + listener.onFailure(exp); } @Override @@ -212,17 +222,6 @@ public class SyncedFlushService extends AbstractComponent { return ThreadPool.Names.SAME; } }); - try { - if (latch.await(inflightOpsTimeout.millis(), TimeUnit.MILLISECONDS) == false) { - logger.debug("{} in flight operation check timed out after [{}]", shardId, syncTimeout); - } - } catch (InterruptedException e) { - logger.debug("{} interrupted while waiting for in flight operation check", shardId); - Thread.currentThread().interrupt(); - } - final int count = result.get(); - logger.trace("{} in flight operation count [{}]", shardId, count); - return count; } @@ -286,15 +285,17 @@ public class SyncedFlushService extends AbstractComponent { /** * send presync requests to all started copies of the given shard */ - Map sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId) { - final CountDownLatch countDownLatch = new CountDownLatch(shards.size()); + void sendPreSyncRequests(final List shards, final ClusterState state, final ShardId shardId, final ActionListener> listener) { + final CountDown countDown = new CountDown(shards.size()); final ConcurrentMap commitIds = ConcurrentCollections.newConcurrentMap(); for (final ShardRouting shard : shards) { logger.trace("{} sending pre-synced flush request to {}", shardId, shard); final DiscoveryNode node = state.nodes().get(shard.currentNodeId()); if (node == null) { logger.trace("{} shard routing {} refers to an unknown node. skipping.", shardId, shard); - countDownLatch.countDown(); + if(countDown.countDown()) { + listener.onResponse(commitIds); + } continue; } transportService.sendRequest(node, PRE_SYNCED_FLUSH_ACTION_NAME, new PreSyncedFlushRequest(shard.shardId()), new BaseTransportResponseHandler() { @@ -308,13 +309,17 @@ public class SyncedFlushService extends AbstractComponent { Engine.CommitId existing = commitIds.putIfAbsent(node.id(), response.commitId()); assert existing == null : "got two answers for node [" + node + "]"; // count after the assert so we won't decrement twice in handleException - countDownLatch.countDown(); + if(countDown.countDown()) { + listener.onResponse(commitIds); + } } @Override public void handleException(TransportException exp) { logger.trace("{} error while performing pre synced flush on [{}], skipping", shardId, exp, shard); - countDownLatch.countDown(); + if(countDown.countDown()) { + listener.onResponse(commitIds); + } } @Override @@ -323,16 +328,6 @@ public class SyncedFlushService extends AbstractComponent { } }); } - try { - if (countDownLatch.await(preSyncTimeout.millis(), TimeUnit.MILLISECONDS) == false) { - logger.debug("{} waiting for pre sync flush requests timed out after [{}]. pending ops [{}]", shardId, preSyncTimeout, countDownLatch.getCount()); - } - } catch (InterruptedException e) { - logger.debug("{} interrupted while waiting for presync requests", shardId); - Thread.currentThread().interrupt(); - } - - return commitIds; } private PreSyncedFlushResponse performPreSyncedFlush(PreSyncedFlushRequest request) { diff --git a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java index 6ee3448b99f..93ead8ac862 100644 --- a/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java +++ b/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.settings.ImmutableSettings; @@ -40,7 +39,7 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.InternalTestCluster.RestartCallback; -import org.elasticsearch.test.SyncedFlushUtil; +import org.elasticsearch.indices.SyncedFlushUtil; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/indices/FlushTest.java b/src/test/java/org/elasticsearch/indices/FlushTest.java index d66100218a7..65fb2f6816b 100644 --- a/src/test/java/org/elasticsearch/indices/FlushTest.java +++ b/src/test/java/org/elasticsearch/indices/FlushTest.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.elasticsearch.test.SyncedFlushUtil; import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java index beb55f6ff4e..348c5753c83 100644 --- a/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java +++ b/src/test/java/org/elasticsearch/indices/SycnedFlushSingleNodeTest.java @@ -28,7 +28,6 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchSingleNodeTest; -import org.elasticsearch.test.SyncedFlushUtil; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import java.util.List; @@ -50,11 +49,11 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); client().prepareIndex("test", "test", "2").setSource("{}").get(); String syncId = Strings.base64UUID(); - SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); listener.latch.await(); assertNull(listener.error); @@ -67,8 +66,8 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { assertFalse(syncedFlushResult.shardResponses().get(activeShards.get(0)).success()); assertEquals("pending operations", syncedFlushResult.shardResponses().get(activeShards.get(0)).failureReason()); - flushService.sendPreSyncRequests(activeShards, state, shardId); // pull another commit and make sure we can't seal with the old one - listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); // pull another commit and make sure we can't seal with the old one + listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId,listener); listener.latch.await(); assertNull(listener.error); @@ -91,7 +90,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); final ShardId shardId = shard.shardId(); - SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.attemptSyncedFlush(shardId, listener); listener.latch.await(); assertNull(listener.error); @@ -114,7 +113,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final ShardId shardId = shard.shardId(); shard.incrementOperationCounter(); try { - SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.attemptSyncedFlush(shardId, listener); listener.latch.await(); assertNull(listener.error); @@ -135,7 +134,7 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { IndexShard shard = test.shard(0); SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); - SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.attemptSyncedFlush(new ShardId("test", 1), listener); listener.latch.await(); assertNotNull(listener.error); @@ -145,14 +144,14 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final ShardId shardId = shard.shardId(); client().admin().indices().prepareClose("test").get(); - listener = new SyncedFlushUtil.SyncResultListener(); + listener = new SyncedFlushUtil.LatchedListener(); flushService.attemptSyncedFlush(shardId, listener); listener.latch.await(); assertNotNull(listener.error); assertNull(listener.result); assertEquals("closed", listener.error.getMessage()); - listener = new SyncedFlushUtil.SyncResultListener(); + listener = new SyncedFlushUtil.LatchedListener(); flushService.attemptSyncedFlush(new ShardId("nosuchindex", 0), listener); listener.latch.await(); assertNotNull(listener.error); @@ -172,14 +171,14 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); if (randomBoolean()) { client().prepareIndex("test", "test", "2").setSource("{}").get(); } client().admin().indices().prepareFlush("test").setForce(true).get(); String syncId = Strings.base64UUID(); - final SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + final SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); listener.latch.await(); assertNull(listener.error); @@ -206,11 +205,11 @@ public class SycnedFlushSingleNodeTest extends ElasticsearchSingleNodeTest { final IndexShardRoutingTable shardRoutingTable = flushService.getActiveShardRoutings(shardId, state); final List activeShards = shardRoutingTable.activeShards(); assertEquals("exactly one active shard", 1, activeShards.size()); - Map commitIds = flushService.sendPreSyncRequests(activeShards, state, shardId); + Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); commitIds.clear(); // wipe it... String syncId = Strings.base64UUID(); - SyncedFlushUtil.SyncResultListener listener = new SyncedFlushUtil.SyncResultListener(); + SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, listener); listener.latch.await(); assertNull(listener.error); diff --git a/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java b/src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java similarity index 60% rename from src/test/java/org/elasticsearch/test/SyncedFlushUtil.java rename to src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java index ddf29297851..e16c85b4b7e 100644 --- a/src/test/java/org/elasticsearch/test/SyncedFlushUtil.java +++ b/src/test/java/org/elasticsearch/indices/SyncedFlushUtil.java @@ -16,15 +16,20 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.test; +package org.elasticsearch.indices; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.SyncedFlushService; +import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicReference; /** Utils for SyncedFlush */ public class SyncedFlushUtil { @@ -37,7 +42,7 @@ public class SyncedFlushUtil { * Blocking version of {@link SyncedFlushService#attemptSyncedFlush(ShardId, ActionListener)} */ public static SyncedFlushService.SyncedFlushResult attemptSyncedFlush(SyncedFlushService service, ShardId shardId) { - SyncResultListener listener = new SyncResultListener(); + LatchedListener listener = new LatchedListener(); service.attemptSyncedFlush(shardId, listener); try { listener.latch.await(); @@ -50,14 +55,14 @@ public class SyncedFlushUtil { return listener.result; } - public static final class SyncResultListener implements ActionListener { - public volatile SyncedFlushService.SyncedFlushResult result; + public static final class LatchedListener implements ActionListener { + public volatile T result; public volatile Throwable error; public final CountDownLatch latch = new CountDownLatch(1); @Override - public void onResponse(SyncedFlushService.SyncedFlushResult syncedFlushResult) { - result = syncedFlushResult; + public void onResponse(T result) { + this.result = result; latch.countDown(); } @@ -68,4 +73,21 @@ public class SyncedFlushUtil { } } + /** + * Blocking version of {@link SyncedFlushService#sendPreSyncRequests(List, ClusterState, ShardId, ActionListener)} + */ + public static Map sendPreSyncRequests(SyncedFlushService service, List activeShards, ClusterState state, ShardId shardId) { + LatchedListener> listener = new LatchedListener<>(); + service.sendPreSyncRequests(activeShards, state, shardId, listener); + try { + listener.latch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (listener.error != null) { + throw ExceptionsHelper.convertToElastic(listener.error); + } + return listener.result; + } + } From 488be75d1982ee7f1debf8b31bc498f0306464a8 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 12:26:08 +0200 Subject: [PATCH 36/38] Add some words about the purpose of a seal etc. --- docs/reference/indices/seal.asciidoc | 4 ++++ .../java/org/elasticsearch/index/engine/InternalEngine.java | 2 -- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/reference/indices/seal.asciidoc b/docs/reference/indices/seal.asciidoc index 03de367af71..583888617b5 100644 --- a/docs/reference/indices/seal.asciidoc +++ b/docs/reference/indices/seal.asciidoc @@ -5,6 +5,10 @@ The seal API allows to flush one or more indices and adds a marker to primaries The seal marker is used during recovery after a node restarts. If a replica is allocated on a node which already has a shard copy with the same seal as the primary then no files will be copied during recovery. Sealing is a best effort operation. If write operations are ongoing while the sealing is in progress then writing the seal might fail on some copies. +A seal marks a point in time snapshot (a low level lucene commit). This mark can be used to decide if the initial, rather resource heavy, recovery phase where segments or event the entire lucene index is copied over the network can +be skipped. If the indices on both sides of the recover have the same seal no segment files need to be copied and transaction log replay can start immediately. The seal breaks as soon as the shard issues a new lucene commit, uncommitted operations +in the transaction log do not break the seal until they are committed. + [source,js] -------------------------------------------------- $ curl -XPOST 'http://localhost:9200/twitter/_seal' diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1bb2bfc5709..5ab98d44763 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -750,7 +750,6 @@ public class InternalEngine extends Engine { try { if (flushNeeded || force) { flushNeeded = false; - final long translogId; try { translog.prepareCommit(); logger.trace("starting commit for flush; commitTranslog=true"); @@ -759,7 +758,6 @@ public class InternalEngine extends Engine { translog.commit(); // we need to refresh in order to clear older version values refresh("version_table_flush"); - } catch (Throwable e) { throw new FlushFailedEngineException(shardId, e); } From 2439faaf5be8d463a0add6468440e41017950f91 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 14:43:03 +0200 Subject: [PATCH 37/38] [TEST] Add more engine unittests for sync IDs --- .../index/engine/InternalEngineTests.java | 58 ++++++++++++++++--- 1 file changed, 50 insertions(+), 8 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index d352d6334e0..fa8c0f3be70 100644 --- a/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -700,16 +700,58 @@ public class InternalEngineTests extends ElasticsearchTestCase { byte[] wrongBytes = Base64.decode(commitID.toString()); wrongBytes[0] = (byte) ~wrongBytes[0]; Engine.CommitId wrongId = new Engine.CommitId(wrongBytes); - assertThat("should fail to sync flush with wrong id (but no docs)", engine.syncFlush(syncId + "1", wrongId), - equalTo(Engine.SyncedFlushResult.COMMIT_MISMATCH)); + assertEquals("should fail to sync flush with wrong id (but no docs)", engine.syncFlush(syncId + "1", wrongId), + Engine.SyncedFlushResult.COMMIT_MISMATCH); engine.create(new Engine.Create(null, newUid("2"), doc)); - assertThat("should fail to sync flush with right id but pending doc", engine.syncFlush(syncId + "2", commitID), - equalTo(Engine.SyncedFlushResult.PENDING_OPERATIONS)); + assertEquals("should fail to sync flush with right id but pending doc", engine.syncFlush(syncId + "2", commitID), + Engine.SyncedFlushResult.PENDING_OPERATIONS); commitID = engine.flush(); - assertThat("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), - equalTo(Engine.SyncedFlushResult.SUCCESS)); - assertThat(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); - assertThat(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), equalTo(syncId)); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + } + + public void testSycnedFlushSurvivesEngineRestart() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + engine.create(new Engine.Create(null, newUid("1"), doc)); + final Engine.CommitId commitID = engine.flush(); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + EngineConfig config = engine.config(); + if (randomBoolean()) { + engine.close(); + } else { + engine.flushAndClose(); + } + engine = new InternalEngine(config, randomBoolean()); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + } + + public void testSycnedFlushVanishesOnReplay() throws IOException { + final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + engine.create(new Engine.Create(null, newUid("1"), doc)); + final Engine.CommitId commitID = engine.flush(); + assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID), + Engine.SyncedFlushResult.SUCCESS); + assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + doc = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), new BytesArray("{}"), null); + engine.create(new Engine.Create(null, newUid("2"), doc)); + EngineConfig config = engine.config(); + engine.close(); + final MockDirectoryWrapper directory = DirectoryUtils.getLeaf(store.directory(), MockDirectoryWrapper.class); + if (directory != null) { + // since we rollback the IW we are writing the same segment files again after starting IW but MDW prevents + // this so we have to disable the check explicitly + directory.setPreventDoubleWrite(false); + } + engine = new InternalEngine(config, false); + assertNull("Sync ID must be gone since we have a document to replay", engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID)); } @Test From 78d7e5b6957719304a154807e1aae0f84acd7f18 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 19 May 2015 14:53:39 +0200 Subject: [PATCH 38/38] Minor cleanups in SyncedFlushService --- .../elasticsearch/indices/SyncedFlushService.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java index fc0a1b50df1..1821da0e2bc 100644 --- a/src/main/java/org/elasticsearch/indices/SyncedFlushService.java +++ b/src/main/java/org/elasticsearch/indices/SyncedFlushService.java @@ -59,9 +59,9 @@ import java.util.concurrent.atomic.AtomicInteger; public class SyncedFlushService extends AbstractComponent { - public static final String PRE_SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/pre"; - public static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; - public static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; + private static final String PRE_SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/pre"; + private static final String SYNCED_FLUSH_ACTION_NAME = "internal:indices/flush/synced/sync"; + private static final String IN_FLIGHT_OPS_ACTION_NAME = "internal:indices/flush/synced/in_flight"; private final IndicesService indicesService; private final ClusterService clusterService; @@ -693,7 +693,6 @@ public class SyncedFlushService extends AbstractComponent { int opCount; - public InFlightOpsResponse() { } @@ -725,8 +724,7 @@ public class SyncedFlushService extends AbstractComponent { } } - - private class PreSyncedFlushTransportHandler implements TransportRequestHandler { + private final class PreSyncedFlushTransportHandler implements TransportRequestHandler { @Override public void messageReceived(PreSyncedFlushRequest request, TransportChannel channel) throws Exception { @@ -734,8 +732,7 @@ public class SyncedFlushService extends AbstractComponent { } } - - private class SyncedFlushTransportHandler implements TransportRequestHandler { + private final class SyncedFlushTransportHandler implements TransportRequestHandler { @Override public void messageReceived(SyncedFlushRequest request, TransportChannel channel) throws Exception { @@ -743,7 +740,7 @@ public class SyncedFlushService extends AbstractComponent { } } - private class InFlightOpCountTransportHandler implements TransportRequestHandler { + private final class InFlightOpCountTransportHandler implements TransportRequestHandler { @Override public void messageReceived(InFlightOpsRequest request, TransportChannel channel) throws Exception {