opensearch-project / OpenSearch

🔎 Open source distributed and RESTful search engine.
https://opensearch.org/docs/latest/opensearch/index/
Apache License 2.0
9.88k stars 1.84k forks source link

[BUG] IO Freeze on Leader cause cluster publication to get stuck #1165

Open Bukhtawar opened 3 years ago

Bukhtawar commented 3 years ago

Describe the bug

The publication of cluster state is time bound to 30s by a cluster.publish.timeout settings. If this time is reached before the new cluster state is committed then the cluster state change is rejected and the leader considers itself to have failed. It stands down and starts trying to elect a new master.

There is a bug in leader that when it tries to publish the new cluster state it first tries acquire a lock(0x0000000097a2f970) to flush the new state under a mutex to disk. The same lock(0x0000000097a2f970) is used to cancel the publication on timeout. Below is the state of the timeout scheduler meant to cancel the publication. So essentially if the flushing of cluster state is stuck on IO, so will the cancellation of the publication since both of them share the same mutex. So leader will not step down and effectively block the cluster from making progress

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27968]" #122622 daemon prio=5 os_prio=0 cpu=305.52ms elapsed=1415.72s tid=0x00007f12d842f000 nid=0x5ac5 waiting for monitor entry  [0x00007f12812d1000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator$CoordinatorPublication$2.run(Coordinator.java:1335)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:642)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

FS Health checks at this point

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27998]" #122744 daemon prio=5 os_prio=0 cpu=4.93ms elapsed=83.04s tid=0x00007f12d8751000 nid=0x7737 runnable  [0x00007f128937c000]
   java.lang.Thread.State: RUNNABLE
    at sun.nio.ch.FileDispatcherImpl.force0(java.base@11.0.6/Native Method)
    at sun.nio.ch.FileDispatcherImpl.force(java.base@11.0.6/FileDispatcherImpl.java:82)
    at sun.nio.ch.FileChannelImpl.force(java.base@11.0.6/FileChannelImpl.java:461)
    at org.elasticsearch.core.internal.io.IOUtils.fsync(IOUtils.java:278)
    at org.elasticsearch.monitor.fs.FsHealthService$FsHealthMonitor.monitorFSHealth(FsHealthService.java:173)
    at org.elasticsearch.monitor.fs.FsHealthService$FsHealthMonitor.run(FsHealthService.java:146)
    at org.elasticsearch.threadpool.Scheduler$ReschedulingRunnable.doRun(Scheduler.java:223)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:701)
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)
[2021-08-27T01:00:55,775][WARN ][o.e.m.f.FsHealthService  ] [6e913fcd73eacb190aa137e551d85dc6] health check of [var/data/nodes/0] took [10295ms] which is above the warn threshold of [5s]
[2021-08-27T01:04:24,251][WARN ][o.e.m.f.FsHealthService  ] [6e913fcd73eacb190aa137e551d85dc6] health check of [var/data/nodes/0] took [86111ms] which is above the warn threshold of [5s]
[2021-08-27T01:09:12,123][WARN ][o.e.m.f.FsHealthService  ] [6e913fcd73eacb190aa137e551d85dc6] health check of [var/data/nodes/0] took [100075ms] which is above the warn threshold of [5s]
[2021-08-27T01:13:42,773][WARN ][o.e.m.f.FsHealthService  ] [6e913fcd73eacb190aa137e551d85dc6] health check of [var/data/nodes/0] took [145478ms] which is above the warn threshold of [5s]

Leader trying to commit the new cluster state to disk causing other operations to be stalled on the same mutex.

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27981]" #122676 daemon prio=5 os_prio=0 cpu=156.25ms elapsed=879.32s tid=0x00007f12d86b8800 nid=0x6595 runnable  [0x00007f1283af8000]
   java.lang.Thread.State: RUNNABLE
    at sun.nio.fs.UnixNativeDispatcher.stat0(java.base@11.0.6/Native Method)
    at sun.nio.fs.UnixNativeDispatcher.stat(java.base@11.0.6/UnixNativeDispatcher.java:301)
    at sun.nio.fs.UnixFileAttributes.get(java.base@11.0.6/UnixFileAttributes.java:70)
    at sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(java.base@11.0.6/UnixFileAttributeViews.java:52)
    at sun.nio.fs.UnixFileSystemProvider.readAttributes(java.base@11.0.6/UnixFileSystemProvider.java:149)
    at sun.nio.fs.LinuxFileSystemProvider.readAttributes(java.base@11.0.6/LinuxFileSystemProvider.java:99)
    at java.nio.file.Files.readAttributes(java.base@11.0.6/Files.java:1763)
    at java.nio.file.Files.size(java.base@11.0.6/Files.java:2380)
    at org.apache.lucene.store.FSDirectory.fileLength(FSDirectory.java:242)
    at org.apache.lucene.index.SegmentCommitInfo.sizeInBytes(SegmentCommitInfo.java:217)
    at org.apache.lucene.index.MergePolicy.size(MergePolicy.java:558)
    at org.apache.lucene.index.MergePolicy.useCompoundFile(MergePolicy.java:540)
    at org.apache.lucene.index.IndexWriter.mergeMiddle(IndexWriter.java:4518)
    - locked <0x0000000097051bd8> (a org.apache.lucene.index.IndexWriter)
    at org.apache.lucene.index.IndexWriter.merge(IndexWriter.java:4057)
    at org.apache.lucene.index.SerialMergeScheduler.merge(SerialMergeScheduler.java:40)
    - locked <0x0000000097a2fc28> (a org.apache.lucene.index.SerialMergeScheduler)
    at org.apache.lucene.index.IndexWriter.maybeMerge(IndexWriter.java:2158)
    at org.apache.lucene.index.IndexWriter.flush(IndexWriter.java:3567)
    at org.apache.lucene.index.IndexWriter.flush(IndexWriter.java:3545)
    at org.elasticsearch.gateway.PersistedClusterStateService$MetaDataIndexWriter.flush(PersistedClusterStateService.java:516)
    at org.elasticsearch.gateway.PersistedClusterStateService$Writer.updateMetaData(PersistedClusterStateService.java:689)
    at org.elasticsearch.gateway.PersistedClusterStateService$Writer.writeIncrementalStateAndCommit(PersistedClusterStateService.java:615)
    at org.elasticsearch.gateway.GatewayMetaState$LucenePersistedState.setLastAcceptedState(GatewayMetaState.java:543)
    at org.elasticsearch.cluster.coordination.CoordinationState.handlePublishRequest(CoordinationState.java:357)
    at org.elasticsearch.cluster.coordination.Coordinator.handlePublishRequest(Coordinator.java:353)
    - locked <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$2310/0x0000000100c78040.apply(Unknown Source)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler.acceptState(PublicationTransportHandler.java:449)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler.handleIncomingPublishRequest(PublicationTransportHandler.java:432)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler.lambda$new$0(PublicationTransportHandler.java:100)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler$$Lambda$2313/0x0000000100c78c40.messageReceived(Unknown Source)
    ......
    at org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(RequestHandlerRegistry.java:63)
    at org.elasticsearch.transport.TransportService$7.doRun(TransportService.java:767)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:701)
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

Note other processing like Follower Checker remove node is stuck on the same mutex


"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27983]" #122679 daemon prio=5 os_prio=0 cpu=85.98ms elapsed=876.81s tid=0x00007f128411d800 nid=0x6599 waiting for monitor entry  [0x00007f12946cb000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator$CoordinatorPublication$3.run(Coordinator.java:1348)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:642)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27987]" #122698 daemon prio=5 os_prio=0 cpu=6.49ms elapsed=426.07s tid=0x00007f12d8428000 nid=0x705a waiting for monitor entry  [0x00007f1283dfc000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator.removeNode(Coordinator.java:248)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$2329/0x0000000100c7e440.accept(Unknown Source)
    at org.elasticsearch.cluster.coordination.FollowersChecker$FollowerChecker$2.run(FollowersChecker.java:454)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:642)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

   Locked ownable synchronizers:
    - <0x00000000804fb130> (a java.util.concurrent.ThreadPoolExecutor$Worker)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27988]" #122699 daemon prio=5 os_prio=0 cpu=0.25ms elapsed=423.34s tid=0x00007f127e36d800 nid=0x705b waiting for monitor entry  [0x00007f1282ae9000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator$5.onResponse(Coordinator.java:1134)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler$3.handleResponse(PublicationTransportHandler.java:286)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler$3.handleResponse(PublicationTransportHandler.java:277)
    at org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler.handleResponse(TransportService.java:1129)
    at org.elasticsearch.transport.InboundHandler$1.doRun(InboundHandler.java:222)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:701)
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

   Locked ownable synchronizers:
    - <0x000000008012f990> (a java.util.concurrent.ThreadPoolExecutor$Worker)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27989]" #122700 daemon prio=5 os_prio=0 cpu=6.40ms elapsed=423.33s tid=0x00007f12d842d000 nid=0x705c waiting for monitor entry  [0x00007f1289079000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator.removeNode(Coordinator.java:248)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$2329/0x0000000100c7e440.accept(Unknown Source)
    at org.elasticsearch.cluster.coordination.FollowersChecker$FollowerChecker$2.run(FollowersChecker.java:454)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:642)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

   Locked ownable synchronizers:
    - <0x00000000804fb590> (a java.util.concurrent.ThreadPoolExecutor$Worker)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27990]" #122701 daemon prio=5 os_prio=0 cpu=0.92ms elapsed=402.18s tid=0x00007f12d874e800 nid=0x715b waiting for monitor entry  [0x00007f128d58e000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator$5.onResponse(Coordinator.java:1134)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler$3.handleResponse(PublicationTransportHandler.java:286)
    at org.elasticsearch.cluster.coordination.PublicationTransportHandler$3.handleResponse(PublicationTransportHandler.java:277)
    at org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler.handleResponse(TransportService.java:1129)
    at org.elasticsearch.transport.InboundHandler$1.doRun(InboundHandler.java:222)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:701)
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27993]" #122705 daemon prio=5 os_prio=0 cpu=27.30ms elapsed=373.33s tid=0x00007f12d8750800 nid=0x71a7 waiting for monitor entry  [0x00007f12822e0000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator.getStateForMasterService(Coordinator.java:1040)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.Coordinator.lambda$handleJoinRequest$7(Coordinator.java:500)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$3288/0x0000000101881c40.accept(Unknown Source)
    at org.elasticsearch.action.ActionListener$1.onResponse(ActionListener.java:63)
    at org.elasticsearch.transport.ClusterConnectionManager.connectToNode(ClusterConnectionManager.java:120)
    at org.elasticsearch.transport.TransportService.connectToNode(TransportService.java:375)
    at org.elasticsearch.transport.TransportService.connectToNode(TransportService.java:359)
    at org.elasticsearch.cluster.coordination.Coordinator.handleJoinRequest(Coordinator.java:499)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$2285/0x0000000100c71c40.accept(Unknown Source)
    at org.elasticsearch.cluster.coordination.JoinHelper.lambda$new$0(JoinHelper.java:136)
    at org.elasticsearch.cluster.coordination.JoinHelper$$Lambda$2288/0x0000000100c72840.messageReceived(Unknown Source)
.......
    at org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(RequestHandlerRegistry.java:63)
    at org.elasticsearch.transport.InboundHandler$RequestHandler.doRun(InboundHandler.java:264)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:701)
    at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

   Locked ownable synchronizers:
    - <0x0000000080be46b8> (a java.util.concurrent.ThreadPoolExecutor$Worker)

"elasticsearch[6e913fcd73eacb190aa137e551d85dc6][generic][T#27994]" #122706 daemon prio=5 os_prio=0 cpu=29.90ms elapsed=342.92s tid=0x00007f12d8781800 nid=0x722a waiting for monitor entry  [0x00007f12819d8000]
   java.lang.Thread.State: BLOCKED (on object monitor)
    at org.elasticsearch.cluster.coordination.Coordinator.removeNode(Coordinator.java:248)
    - waiting to lock <0x0000000097a2f970> (a java.lang.Object)
    at org.elasticsearch.cluster.coordination.Coordinator$$Lambda$2329/0x0000000100c7e440.accept(Unknown Source)
    at org.elasticsearch.cluster.coordination.FollowersChecker$FollowerChecker$2.run(FollowersChecker.java:454)
    at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run(ThreadContext.java:642)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.6/ThreadPoolExecutor.java:1128)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.6/ThreadPoolExecutor.java:628)
    at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

Proposal

  1. Enforce a timeout on leader to commit a cluster state including the time it takes it commit to fsync the state.
  2. Improve health checks to fail the leader or any node for that master if the Fs Health check takes beyond 60s

To Reproduce Steps to reproduce the behavior:

  1. Go to '...'
  2. Click on '....'
  3. Scroll down to '....'
  4. See error

Expected behavior A clear and concise description of what you expected to happen.

Plugins Please list all plugins currently enabled.

Screenshots If applicable, add screenshots to help explain your problem.

Host/Environment (please complete the following information):

Additional context Add any other context about the problem here.

itiyamas commented 3 years ago

Can you add the steps to reproduce this issue or write the test case for the same?

itiyamas commented 3 years ago

I don't see the FS Health failing fixing this problem as once the health check fails, the onLeaderFailure will block again on the same mutex. We need to somehow add timeouts to cluster state persistence calls. I believe once that is fixed, this issue will go way, irrespective of FS health check on stuck IO (Essentially your point 1 in Proposal above). This change may be tricky to achieve. FS health check on stuck IO is a good change in itself, but not a fix for this issue.

Bukhtawar commented 3 years ago

Fs health checks does pro-active checks to identify a bad node and evict the same from the cluster, rather than waiting for a cluster state update to remove the stuck leader. You rightly pointed out that this fix itself is insufficient since the mutex for onLeaderFailed will still get BLOCKED, which is what the first proposal tries to fix. Moreover Fs health checks would help with situations in general across data nodes.

anasalkouz commented 3 years ago

Hi Bukhtawar, are you actively working on this issue? I see a pull request #1167 already submitted. anything else is pending before we can close the issue?

andrross commented 3 years ago

We don't seem to have any metrics on how frequently this issue manifests, but it appears to happen when the filesystem in use is not a local disk (i.e. EBS).

anasalkouz commented 2 years ago

@Bukhtawar is there anything else pending on this issue before we close it?