Переглянути джерело

Abort pending deletion on IndicesService stop (#123569)

When IndicesService is closed, the pending deletion may still be in
progress due to indices removed before IndicesService gets closed. If
the deletion stucks for some reason, it can stall the node shutdown.
This PR aborts the pending deletion more promptly by not retry after
IndicesService is stopped.

Resolves: #121717 Resolves: #121716  Resolves: #122119
Yang Wang 7 місяців тому
батько
коміт
c7e7dbe904

+ 5 - 0
docs/changelog/123569.yaml

@@ -0,0 +1,5 @@
+pr: 123569
+summary: Abort pending deletion on `IndicesService` close
+area: Store
+type: enhancement
+issues: []

+ 0 - 8
muted-tests.yml

@@ -254,8 +254,6 @@ tests:
 - class: org.elasticsearch.test.rest.ClientYamlTestSuiteIT
   method: test {yaml=snapshot.delete/10_basic/Delete a snapshot asynchronously}
   issue: https://github.com/elastic/elasticsearch/issues/122102
-- class: org.elasticsearch.datastreams.TSDBPassthroughIndexingIT
-  issue: https://github.com/elastic/elasticsearch/issues/121716
 - class: org.elasticsearch.smoketest.SmokeTestMonitoringWithSecurityIT
   method: testHTTPExporterWithSSL
   issue: https://github.com/elastic/elasticsearch/issues/122220
@@ -286,15 +284,9 @@ tests:
 - class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT
   method: test {yaml=reference/snapshot-restore/apis/get-snapshot-api/line_408}
   issue: https://github.com/elastic/elasticsearch/issues/122681
-- class: org.elasticsearch.xpack.autoscaling.storage.ReactiveStorageIT
-  method: testScaleWhileShrinking
-  issue: https://github.com/elastic/elasticsearch/issues/122119
 - class: org.elasticsearch.search.basic.SearchWithRandomDisconnectsIT
   method: testSearchWithRandomDisconnects
   issue: https://github.com/elastic/elasticsearch/issues/122707
-- class: org.elasticsearch.snapshots.DedicatedClusterSnapshotRestoreIT
-  method: testRestoreShrinkIndex
-  issue: https://github.com/elastic/elasticsearch/issues/121717
 - class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT
   method: test {yaml=reference/cat/allocation/cat-allocation-example}
   issue: https://github.com/elastic/elasticsearch/issues/121976

+ 11 - 1
server/src/main/java/org/elasticsearch/indices/IndicesService.java

@@ -255,6 +255,7 @@ public class IndicesService extends AbstractLifecycleComponent
     private final Map<String, IndexStorePlugin.RecoveryStateFactory> recoveryStateFactories;
     private final IndexStorePlugin.IndexFoldersDeletionListener indexFoldersDeletionListeners;
     final AbstractRefCounted indicesRefCount; // pkg-private for testing
+    private final CountDownLatch stopLatch = new CountDownLatch(1);
     private final CountDownLatch closeLatch = new CountDownLatch(1);
     private volatile boolean idFieldDataEnabled;
     private volatile boolean allowExpensiveQueries;
@@ -403,6 +404,7 @@ public class IndicesService extends AbstractLifecycleComponent
 
     @Override
     protected void doStop() {
+        stopLatch.countDown();
         clusterService.removeApplier(timestampFieldMapperService);
         timestampFieldMapperService.doStop();
 
@@ -1440,7 +1442,15 @@ public class IndicesService extends AbstractLifecycleComponent
                     }
                     if (remove.isEmpty() == false) {
                         logger.warn("{} still pending deletes present for shards {} - retrying", index, remove.toString());
-                        Thread.sleep(sleepTime);
+                        if (stopLatch.await(sleepTime, TimeUnit.MILLISECONDS)) {
+                            logger.info(
+                                "Indices service stopped. {} aborting pending deletes after [{}] for shards {}",
+                                index,
+                                TimeValue.timeValueNanos(System.nanoTime() - startTimeNS),
+                                remove.toString()
+                            );
+                            break;
+                        }
                         sleepTime = Math.min(maxSleepTimeMs, sleepTime * 2); // increase the sleep time gradually
                         logger.debug("{} schedule pending delete retry after {} ms", index, sleepTime);
                     }