2626import org .elasticsearch .action .admin .indices .flush .FlushRequest ;
2727import org .elasticsearch .action .index .IndexRequest ;
2828import org .elasticsearch .action .support .PlainActionFuture ;
29+ import org .elasticsearch .cluster .metadata .IndexMetaData ;
2930import org .elasticsearch .cluster .node .DiscoveryNode ;
3031import org .elasticsearch .cluster .routing .RecoverySource ;
3132import org .elasticsearch .cluster .routing .ShardRoutingHelper ;
32- import org .elasticsearch .common .CheckedFunction ;
3333import org .elasticsearch .common .Randomness ;
3434import org .elasticsearch .common .UUIDs ;
3535import org .elasticsearch .common .bytes .BytesArray ;
36+ import org .elasticsearch .common .settings .Settings ;
3637import org .elasticsearch .common .xcontent .XContentType ;
38+ import org .elasticsearch .index .engine .NoOpEngine ;
3739import org .elasticsearch .index .mapper .SourceToParse ;
40+ import org .elasticsearch .index .seqno .SeqNoStats ;
3841import org .elasticsearch .index .seqno .SequenceNumbers ;
3942import org .elasticsearch .index .shard .IndexShard ;
4043import org .elasticsearch .index .shard .IndexShardTestCase ;
@@ -134,23 +137,24 @@ public void testWriteFileChunksConcurrently() throws Exception {
134137 closeShards (sourceShard , targetShard );
135138 }
136139
137- public void testPrepareIndexForPeerRecovery () throws Exception {
138- CheckedFunction <IndexShard , Long , Exception > populateData = shard -> {
139- List <Long > seqNos = LongStream .range (0 , 100 ).boxed ().collect (Collectors .toList ());
140- Randomness .shuffle (seqNos );
141- for (long seqNo : seqNos ) {
142- shard .applyIndexOperationOnReplica (seqNo , 1 , IndexRequest .UNSET_AUTO_GENERATED_TIMESTAMP , false , new SourceToParse (
143- shard .shardId ().getIndexName (), "_doc" , UUIDs .randomBase64UUID (), new BytesArray ("{}" ), XContentType .JSON ));
144- if (randomInt (100 ) < 5 ) {
145- shard .flush (new FlushRequest ().waitIfOngoing (true ));
146- }
140+ private SeqNoStats populateRandomData (IndexShard shard ) throws IOException {
141+ List <Long > seqNos = LongStream .range (0 , 100 ).boxed ().collect (Collectors .toList ());
142+ Randomness .shuffle (seqNos );
143+ for (long seqNo : seqNos ) {
144+ shard .applyIndexOperationOnReplica (seqNo , 1 , IndexRequest .UNSET_AUTO_GENERATED_TIMESTAMP , false , new SourceToParse (
145+ shard .shardId ().getIndexName (), "_doc" , UUIDs .randomBase64UUID (), new BytesArray ("{}" ), XContentType .JSON ));
146+ if (randomInt (100 ) < 5 ) {
147+ shard .flush (new FlushRequest ().waitIfOngoing (true ));
147148 }
148- shard .sync ();
149- long globalCheckpoint = randomLongBetween (SequenceNumbers .NO_OPS_PERFORMED , shard .getLocalCheckpoint ());
150- shard .updateGlobalCheckpointOnReplica (globalCheckpoint , "test" );
151- shard .sync ();
152- return globalCheckpoint ;
153- };
149+ }
150+ shard .sync ();
151+ long globalCheckpoint = randomLongBetween (SequenceNumbers .NO_OPS_PERFORMED , shard .getLocalCheckpoint ());
152+ shard .updateGlobalCheckpointOnReplica (globalCheckpoint , "test" );
153+ shard .sync ();
154+ return shard .seqNoStats ();
155+ }
156+
157+ public void testPrepareIndexForPeerRecovery () throws Exception {
154158 DiscoveryNode localNode = new DiscoveryNode ("foo" , buildNewFakeTransportAddress (),
155159 Collections .emptyMap (), Collections .emptySet (), Version .CURRENT );
156160
@@ -166,7 +170,7 @@ public void testPrepareIndexForPeerRecovery() throws Exception {
166170
167171 // good copy
168172 shard = newStartedShard (false );
169- long globalCheckpoint = populateData . apply (shard );
173+ long globalCheckpoint = populateRandomData (shard ). getGlobalCheckpoint ( );
170174 Optional <SequenceNumbers .CommitInfo > safeCommit = shard .store ().findSafeIndexCommit (globalCheckpoint );
171175 assertTrue (safeCommit .isPresent ());
172176 int expectedTotalLocal = 0 ;
@@ -191,7 +195,7 @@ public void testPrepareIndexForPeerRecovery() throws Exception {
191195 // corrupted copy
192196 shard = newStartedShard (false );
193197 if (randomBoolean ()) {
194- populateData . apply (shard );
198+ populateRandomData (shard );
195199 }
196200 shard .store ().markStoreCorrupted (new IOException ("test" ));
197201 replica = reinitShard (shard , ShardRoutingHelper .initWithSameId (shard .routingEntry (),
@@ -206,7 +210,7 @@ public void testPrepareIndexForPeerRecovery() throws Exception {
206210
207211 // copy with truncated translog
208212 shard = newStartedShard (false );
209- globalCheckpoint = populateData . apply (shard );
213+ globalCheckpoint = populateRandomData (shard ). getGlobalCheckpoint ( );
210214 replica = reinitShard (shard , ShardRoutingHelper .initWithSameId (shard .routingEntry (),
211215 RecoverySource .PeerRecoverySource .INSTANCE ));
212216 String translogUUID = Translog .createEmptyTranslog (replica .shardPath ().resolveTranslog (), globalCheckpoint ,
@@ -226,4 +230,32 @@ public void testPrepareIndexForPeerRecovery() throws Exception {
226230 assertThat (replica .getLastKnownGlobalCheckpoint (), equalTo (UNASSIGNED_SEQ_NO ));
227231 closeShards (replica );
228232 }
233+
234+ public void testClosedIndexSkipsLocalRecovery () throws Exception {
235+ DiscoveryNode localNode = new DiscoveryNode ("foo" , buildNewFakeTransportAddress (),
236+ Collections .emptyMap (), Collections .emptySet (), Version .CURRENT );
237+ IndexShard shard = newStartedShard (false );
238+ long globalCheckpoint = populateRandomData (shard ).getGlobalCheckpoint ();
239+ Optional <SequenceNumbers .CommitInfo > safeCommit = shard .store ().findSafeIndexCommit (globalCheckpoint );
240+ assertTrue (safeCommit .isPresent ());
241+ final IndexMetaData indexMetaData ;
242+ if (randomBoolean ()) {
243+ indexMetaData = IndexMetaData .builder (shard .indexSettings ().getIndexMetaData ())
244+ .settings (shard .indexSettings ().getSettings ())
245+ .state (IndexMetaData .State .CLOSE ).build ();
246+ } else {
247+ indexMetaData = IndexMetaData .builder (shard .indexSettings ().getIndexMetaData ())
248+ .settings (Settings .builder ().put (shard .indexSettings ().getSettings ())
249+ .put (IndexMetaData .SETTING_BLOCKS_WRITE , true )).build ();
250+ }
251+ IndexShard replica = reinitShard (shard , ShardRoutingHelper .initWithSameId (shard .routingEntry (),
252+ RecoverySource .PeerRecoverySource .INSTANCE ), indexMetaData , NoOpEngine ::new );
253+ replica .markAsRecovering ("for testing" , new RecoveryState (replica .routingEntry (), localNode , localNode ));
254+ replica .prepareForIndexRecovery ();
255+ assertThat (replica .recoverLocallyUpToGlobalCheckpoint (), equalTo (safeCommit .get ().localCheckpoint + 1 ));
256+ assertThat (replica .recoveryState ().getTranslog ().totalLocal (), equalTo (0 ));
257+ assertThat (replica .recoveryState ().getTranslog ().recoveredOperations (), equalTo (0 ));
258+ assertThat (replica .getLastKnownGlobalCheckpoint (), equalTo (UNASSIGNED_SEQ_NO ));
259+ closeShards (replica );
260+ }
229261}
0 commit comments