Skip to content

IndicesClusterApplierService blocks applier thread pending completion of a refresh #130839

Open
@DaveCTurner

Description

@DaveCTurner

Seen in an 8.18.3 production cluster.

The applier thread was waiting for the mutex on IndicesClusterStateService.
  at org.elasticsearch.indices.cluster.IndicesClusterStateService.applyClusterState(Lorg/elasticsearch/cluster/ClusterChangedEvent;)V (IndicesClusterStateService.java:255)
  at org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(Lorg/elasticsearch/cluster/ClusterChangedEvent;Lorg/elasticsearch/cluster/service/ClusterApplierRecordingService$Recorder;Ljava/util/Collection;)V (ClusterApplierService.java:544)
  at org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(Lorg/elasticsearch/cluster/ClusterChangedEvent;Lorg/elasticsearch/cluster/service/ClusterApplierRecordingService$Recorder;)V (ClusterApplierService.java:530)
  at org.elasticsearch.cluster.service.ClusterApplierService.applyChanges(Lorg/elasticsearch/cluster/ClusterState;Lorg/elasticsearch/cluster/ClusterState;Ljava/lang/String;Lorg/elasticsearch/cluster/service/ClusterApplierRecordingService$Recorder;)V (ClusterApplierService.java:503)
  at org.elasticsearch.cluster.service.ClusterApplierService.runTask(Ljava/lang/String;Ljava/util/function/Function;Lorg/elasticsearch/action/ActionListener;)V (ClusterApplierService.java:432)
  at org.elasticsearch.cluster.service.ClusterApplierService$UpdateTask.run()V (ClusterApplierService.java:157)
  at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run()V (ThreadContext.java:977)
  at org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.runAndClean(Ljava/lang/Runnable;)V (PrioritizedEsThreadPoolExecutor.java:218)
  at org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run()V (PrioritizedEsThreadPoolExecutor.java:184)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1136)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:635)
  at java.lang.Thread.run()V (Thread.java:840)
The IndicesClusterStateService mutex is held by a thread waiting for Engine#drainForClose to complete.
  at jdk.internal.misc.Unsafe.park(ZJ)V (Native Method)
  at java.util.concurrent.locks.LockSupport.park(Ljava/lang/Object;)V (LockSupport.java:211)
  at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(Ljava/util/concurrent/locks/AbstractQueuedSynchronizer$Node;IZZZJ)I (AbstractQueuedSynchronizer.java:715)
  at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(I)V (AbstractQueuedSynchronizer.java:1047)
  at org.elasticsearch.action.support.PlainActionFuture$Sync.get()Ljava/lang/Object; (PlainActionFuture.java:265)
  at org.elasticsearch.action.support.PlainActionFuture.get()Ljava/lang/Object; (PlainActionFuture.java:96)
  at org.elasticsearch.index.engine.Engine.drainForClose()Z (Engine.java:2035)
  at org.elasticsearch.index.engine.Engine.close()V (Engine.java:2073)
  at org.elasticsearch.core.IOUtils.close(Ljava/io/Closeable;)V (IOUtils.java:71)
  at org.elasticsearch.core.IOUtils.close(Ljava/lang/Exception;[Ljava/io/Closeable;)V (IOUtils.java:87)
  at org.elasticsearch.core.IOUtils.close([Ljava/io/Closeable;)V (IOUtils.java:63)
  at org.elasticsearch.index.shard.IndexShard$4.run()V (IndexShard.java:1775)
  at org.elasticsearch.action.ActionRunnable$1.doRun()V (ActionRunnable.java:37)
  at org.elasticsearch.common.util.concurrent.AbstractRunnable.run()V (AbstractRunnable.java:27)
  at org.elasticsearch.common.util.concurrent.EsExecutors$DirectExecutorService.execute(Ljava/lang/Runnable;)V (EsExecutors.java:316)
  at org.elasticsearch.index.shard.IndexShard.close(Ljava/lang/String;ZLjava/util/concurrent/Executor;Lorg/elasticsearch/action/ActionListener;)V (IndexShard.java:1765)
  at org.elasticsearch.index.IndexService.lambda$closeShard$16(Lorg/elasticsearch/index/shard/IndexShard;Ljava/lang/String;ZLjava/util/concurrent/Executor;Lorg/elasticsearch/index/IndexService$1;)V (IndexService.java:673)
  at org.elasticsearch.index.IndexService$$Lambda$10407+0x000000f801c45c60.accept(Ljava/lang/Object;)V (Unknown Source)
  at org.elasticsearch.action.ActionListener.run(Lorg/elasticsearch/action/ActionListener;Lorg/elasticsearch/core/CheckedConsumer;)V (ActionListener.java:454)
  at org.elasticsearch.index.IndexService.closeShard(Ljava/lang/String;Lorg/elasticsearch/index/shard/ShardId;Lorg/elasticsearch/index/shard/IndexShard;Lorg/elasticsearch/index/store/Store;Lorg/elasticsearch/index/shard/IndexEventListener;Ljava/util/concurrent/Executor;Lorg/elasticsearch/action/ActionListener;)V (IndexService.java:651)
  at org.elasticsearch.index.IndexService.removeShard(ILjava/lang/String;Ljava/util/concurrent/Executor;Lorg/elasticsearch/action/ActionListener;)V (IndexService.java:611)
  at org.elasticsearch.indices.cluster.IndicesClusterStateService.failAndRemoveShard(Lorg/elasticsearch/cluster/routing/ShardRouting;ZLjava/lang/String;Ljava/lang/Exception;Lorg/elasticsearch/cluster/ClusterState;Ljava/util/concurrent/Executor;Lorg/elasticsearch/action/ActionListener;)V (IndicesClusterStateService.java:1032)
  at org.elasticsearch.indices.cluster.IndicesClusterStateService.lambda$handleRecoveryFailure$10(Lorg/elasticsearch/cluster/routing/ShardRouting;ZLjava/lang/Exception;Lorg/elasticsearch/action/ActionListener;)V (IndicesClusterStateService.java:1000)
  at org.elasticsearch.indices.cluster.IndicesClusterStateService$$Lambda$12076+0x000000f801e57868.accept(Ljava/lang/Object;)V (Unknown Source)
  at org.elasticsearch.action.ActionListener.run(Lorg/elasticsearch/action/ActionListener;Lorg/elasticsearch/core/CheckedConsumer;)V (ActionListener.java:454)
  at org.elasticsearch.index.CloseUtils.executeDirectly(Lorg/elasticsearch/core/CheckedConsumer;)V (CloseUtils.java:44)
  at org.elasticsearch.indices.cluster.IndicesClusterStateService.handleRecoveryFailure(Lorg/elasticsearch/cluster/routing/ShardRouting;ZLjava/lang/Exception;)V (IndicesClusterStateService.java:999)
  at org.elasticsearch.indices.cluster.IndicesClusterStateService$RecoveryListener.onRecoveryFailure(Lorg/elasticsearch/indices/recovery/RecoveryFailedException;Z)V (IndicesClusterStateService.java:992)
  at org.elasticsearch.indices.recovery.RecoveryTarget.notifyListener(Lorg/elasticsearch/indices/recovery/RecoveryFailedException;Z)V (RecoveryTarget.java:321)
  at org.elasticsearch.indices.recovery.RecoveryTarget.fail(Lorg/elasticsearch/indices/recovery/RecoveryFailedException;Z)V (RecoveryTarget.java:308)
  at org.elasticsearch.indices.recovery.RecoveriesCollection.failRecovery(JLorg/elasticsearch/indices/recovery/RecoveryFailedException;Z)V (RecoveriesCollection.java:209)
  at org.elasticsearch.indices.recovery.PeerRecoveryTargetService$RecoveryResponseHandler.handleException(Lorg/elasticsearch/transport/TransportException;)V (PeerRecoveryTargetService.java:905)
  at org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler.handleException(Lorg/elasticsearch/transport/TransportException;)V (TransportService.java:1510)
  at org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler.handleException(Lorg/elasticsearch/transport/TransportException;)V (TransportService.java:1510)
  at org.elasticsearch.transport.InboundHandler.doHandleException(Lorg/elasticsearch/transport/TransportResponseHandler;Lorg/elasticsearch/transport/TransportException;)V (InboundHandler.java:476)
  at org.elasticsearch.transport.InboundHandler$3.doRun()V (InboundHandler.java:468)
  at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun()V (ThreadContext.java:1044)
  at org.elasticsearch.common.util.concurrent.AbstractRunnable.run()V (AbstractRunnable.java:27)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1136)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:635)
  at java.lang.Thread.run()V (Thread.java:840)
Engine#drainForClose is waiting for the last reference to Engine#ensureOpenRefs which is held by a thread waiting to acquire ReferenceManager#refreshLock.
  at jdk.internal.misc.Unsafe.park(ZJ)V (Native Method)
  at java.util.concurrent.locks.LockSupport.park(Ljava/lang/Object;)V (LockSupport.java:211)
  at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(Ljava/util/concurrent/locks/AbstractQueuedSynchronizer$Node;IZZZJ)I (AbstractQueuedSynchronizer.java:715)
  at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(I)V (AbstractQueuedSynchronizer.java:938)
  at java.util.concurrent.locks.ReentrantLock$Sync.lock()V (ReentrantLock.java:153)
  at java.util.concurrent.locks.ReentrantLock.lock()V (ReentrantLock.java:322)
  at org.apache.lucene.search.ReferenceManager.maybeRefreshBlocking()V (ReferenceManager.java:238)
  at org.elasticsearch.index.engine.InternalEngine.refresh(Ljava/lang/String;Lorg/elasticsearch/index/engine/Engine$SearcherScope;Z)Lorg/elasticsearch/index/engine/Engine$RefreshResult; (InternalEngine.java:2058)
  at org.elasticsearch.index.engine.InternalEngine.flushHoldingLock(ZZLorg/elasticsearch/action/ActionListener;)V (InternalEngine.java:2237)
  at org.elasticsearch.index.engine.Engine.flush(ZZLorg/elasticsearch/action/ActionListener;)V (Engine.java:1301)
  at org.elasticsearch.index.shard.IndexShard.lambda$flush$8(ZZLorg/elasticsearch/action/ActionListener;)V (IndexShard.java:1497)
  at org.elasticsearch.index.shard.IndexShard$$Lambda$9039+0x000000f801a53818.accept(Ljava/lang/Object;)V (Unknown Source)
  at org.elasticsearch.action.ActionListener.run(Lorg/elasticsearch/action/ActionListener;Lorg/elasticsearch/core/CheckedConsumer;)V (ActionListener.java:454)
  at org.elasticsearch.index.shard.IndexShard.flush(Lorg/elasticsearch/action/admin/indices/flush/FlushRequest;Lorg/elasticsearch/action/ActionListener;)V (IndexShard.java:1489)
  at org.elasticsearch.index.shard.IndexShard.lambda$afterWriteOperation$46()V (IndexShard.java:3922)
  at org.elasticsearch.index.shard.IndexShard$$Lambda$9991+0x000000f801bd3808.run()V (Unknown Source)
  at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run()V (ThreadContext.java:977)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1136)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:635)
  at java.lang.Thread.run()V (Thread.java:840)
ReferenceManager#refreshLock is held by a thread that's busy building a large HNSW graph.
  at org.apache.lucene.internal.vectorization.DefaultVectorUtilSupport.dotProduct([F[F)F (DefaultVectorUtilSupport.java:49)
  at org.apache.lucene.util.VectorUtil.dotProduct([F[F)F (VectorUtil.java:66)
  at org.apache.lucene.index.VectorSimilarityFunction$2.compare([F[F)F (VectorSimilarityFunction.java:55)
  at org.apache.lucene.codecs.hnsw.DefaultFlatVectorScorer$FloatScoringSupplier$1.score(I)F (DefaultFlatVectorScorer.java:149)
  at org.apache.lucene.util.hnsw.HnswGraphSearcher.searchLevel(Lorg/apache/lucene/search/KnnCollector;Lorg/apache/lucene/util/hnsw/RandomVectorScorer;I[ILorg/apache/lucene/util/hnsw/HnswGraph;Lorg/apache/lucene/util/Bits;)V (HnswGraphSearcher.java:237)
  at org.apache.lucene.util.hnsw.HnswGraphBuilder.connectComponents(I)Z (HnswGraphBuilder.java:456)
  at org.apache.lucene.util.hnsw.HnswGraphBuilder.connectComponents()V (HnswGraphBuilder.java:414)
  at org.apache.lucene.util.hnsw.HnswGraphBuilder.finish()V (HnswGraphBuilder.java:407)
  at org.apache.lucene.util.hnsw.HnswGraphBuilder.getCompletedGraph()Lorg/apache/lucene/util/hnsw/OnHeapHnswGraph; (HnswGraphBuilder.java:179)
  at org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsWriter$FieldWriter.getGraph()Lorg/apache/lucene/util/hnsw/OnHeapHnswGraph; (Lucene99HnswVectorsWriter.java:642)
  at org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsWriter.writeField(Lorg/apache/lucene/codecs/lucene99/Lucene99HnswVectorsWriter$FieldWriter;)V (Lucene99HnswVectorsWriter.java:188)
  at org.apache.lucene.codecs.lucene99.Lucene99HnswVectorsWriter.flush(ILorg/apache/lucene/index/Sorter$DocMap;)V (Lucene99HnswVectorsWriter.java:150)
  at org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat$FieldsWriter.flush(ILorg/apache/lucene/index/Sorter$DocMap;)V (PerFieldKnnVectorsFormat.java:115)
  at org.apache.lucene.index.VectorValuesConsumer.flush(Lorg/apache/lucene/index/SegmentWriteState;Lorg/apache/lucene/index/Sorter$DocMap;)V (VectorValuesConsumer.java:76)
  at org.apache.lucene.index.IndexingChain.flush(Lorg/apache/lucene/index/SegmentWriteState;)Lorg/apache/lucene/index/Sorter$DocMap; (IndexingChain.java:296)
  at org.apache.lucene.index.DocumentsWriterPerThread.flush(Lorg/apache/lucene/index/DocumentsWriter$FlushNotifications;)Lorg/apache/lucene/index/DocumentsWriterPerThread$FlushedSegment; (DocumentsWriterPerThread.java:445)
  at org.apache.lucene.index.DocumentsWriter.doFlush(Lorg/apache/lucene/index/DocumentsWriterPerThread;)V (DocumentsWriter.java:502)
  at org.apache.lucene.index.DocumentsWriter.maybeFlush()Z (DocumentsWriter.java:456)
  at org.apache.lucene.index.DocumentsWriter.flushAllThreads()J (DocumentsWriter.java:649)
  at org.apache.lucene.index.IndexWriter.getReader(ZZ)Lorg/apache/lucene/index/DirectoryReader; (IndexWriter.java:579)
  at org.apache.lucene.index.StandardDirectoryReader.doOpenFromWriter(Lorg/apache/lucene/index/IndexCommit;)Lorg/apache/lucene/index/DirectoryReader; (StandardDirectoryReader.java:381)
  at org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged(Lorg/apache/lucene/index/IndexCommit;)Lorg/apache/lucene/index/DirectoryReader; (StandardDirectoryReader.java:355)
  at org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged()Lorg/apache/lucene/index/DirectoryReader; (StandardDirectoryReader.java:345)
  at org.apache.lucene.index.FilterDirectoryReader.doOpenIfChanged()Lorg/apache/lucene/index/DirectoryReader; (FilterDirectoryReader.java:112)
  at org.apache.lucene.index.DirectoryReader.openIfChanged(Lorg/apache/lucene/index/DirectoryReader;)Lorg/apache/lucene/index/DirectoryReader; (DirectoryReader.java:170)
  at org.elasticsearch.index.engine.ElasticsearchReaderManager.refreshIfNeeded(Lorg/elasticsearch/common/lucene/index/ElasticsearchDirectoryReader;)Lorg/elasticsearch/common/lucene/index/ElasticsearchDirectoryReader; (ElasticsearchReaderManager.java:49)
  at org.elasticsearch.index.engine.ElasticsearchReaderManager.refreshIfNeeded(Ljava/lang/Object;)Ljava/lang/Object; (ElasticsearchReaderManager.java:28)
  at org.apache.lucene.search.ReferenceManager.doMaybeRefresh()V (ReferenceManager.java:167)
  at org.apache.lucene.search.ReferenceManager.maybeRefreshBlocking()V (ReferenceManager.java:240)
  at org.elasticsearch.index.engine.InternalEngine.refresh(Ljava/lang/String;Lorg/elasticsearch/index/engine/Engine$SearcherScope;Z)Lorg/elasticsearch/index/engine/Engine$RefreshResult; (InternalEngine.java:2058)
  at org.elasticsearch.index.engine.InternalEngine.refreshInternalSearcher(Ljava/lang/String;Z)Lorg/elasticsearch/index/engine/Engine$RefreshResult; (InternalEngine.java:2038)
  at org.elasticsearch.index.engine.InternalEngine.refreshIfNeeded(Ljava/lang/String;J)V (InternalEngine.java:3263)
  at org.elasticsearch.index.engine.InternalEngine.countChanges(Ljava/lang/String;JJ)I (InternalEngine.java:3123)
  at org.elasticsearch.index.shard.IndexShard.countChanges(Ljava/lang/String;JJ)I (IndexShard.java:2605)
  at org.elasticsearch.indices.recovery.RecoveryTarget.hasUncommittedOperations()Z (RecoveryTarget.java:408)
  at org.elasticsearch.indices.recovery.RecoveryTarget.lambda$finalizeRecovery$2(JJ)Ljava/lang/Void; (RecoveryTarget.java:398)
  at org.elasticsearch.indices.recovery.RecoveryTarget$$Lambda$8951+0x000000f801a1ced0.get()Ljava/lang/Object; (Unknown Source)
  at org.elasticsearch.action.ActionListener.completeWith(Lorg/elasticsearch/action/ActionListener;Lorg/elasticsearch/common/CheckedSupplier;)V (ActionListener.java:356)
  at org.elasticsearch.indices.recovery.RecoveryTarget.finalizeRecovery(JJLorg/elasticsearch/action/ActionListener;)V (RecoveryTarget.java:382)
  at org.elasticsearch.indices.recovery.PeerRecoveryTargetService$5.handleRequest(Lorg/elasticsearch/indices/recovery/RecoveryFinalizeRecoveryRequest;Lorg/elasticsearch/indices/recovery/RecoveryTarget;Lorg/elasticsearch/action/ActionListener;)V (PeerRecoveryTargetService.java:214)
  at org.elasticsearch.indices.recovery.PeerRecoveryTargetService$5.handleRequest(Lorg/elasticsearch/indices/recovery/RecoveryTransportRequest;Lorg/elasticsearch/indices/recovery/RecoveryTarget;Lorg/elasticsearch/action/ActionListener;)V (PeerRecoveryTargetService.java:207)
  at org.elasticsearch.indices.recovery.PeerRecoveryTargetService$RecoveryRequestHandler.messageReceived(Lorg/elasticsearch/indices/recovery/RecoveryTransportRequest;Lorg/elasticsearch/transport/TransportChannel;Lorg/elasticsearch/tasks/Task;)V (PeerRecoveryTargetService.java:691)
  at org.elasticsearch.indices.recovery.PeerRecoveryTargetService$RecoveryRequestHandler.messageReceived(Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/transport/TransportChannel;Lorg/elasticsearch/tasks/Task;)V (PeerRecoveryTargetService.java:678)
  at org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$1.doRun()V (SecurityServerTransportInterceptor.java:579)
  at org.elasticsearch.common.util.concurrent.AbstractRunnable.run()V (AbstractRunnable.java:27)
  at org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(Ljava/lang/Void;)V (SecurityServerTransportInterceptor.java:632)
  at org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(Ljava/lang/Object;)V (SecurityServerTransportInterceptor.java:621)
  at org.elasticsearch.xpack.security.authz.AuthorizationService.authorizeSystemUser(Lorg/elasticsearch/xpack/core/security/authc/Authentication;Ljava/lang/String;Ljava/lang/String;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/action/ActionListener;)V (AuthorizationService.java:685)
  at org.elasticsearch.xpack.security.authz.AuthorizationService.authorize(Lorg/elasticsearch/xpack/core/security/authc/Authentication;Ljava/lang/String;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/action/ActionListener;)V (AuthorizationService.java:317)
  at org.elasticsearch.xpack.security.transport.ServerTransportFilter.lambda$inbound$1(Ljava/lang/String;Lorg/elasticsearch/TransportVersion;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/action/ActionListener;Lorg/elasticsearch/xpack/core/security/authc/Authentication;)V (ServerTransportFilter.java:114)
  at org.elasticsearch.xpack.security.transport.ServerTransportFilter$$Lambda$8220+0x000000f801892208.accept(Ljava/lang/Object;Ljava/lang/Object;)V (Unknown Source)
  at org.elasticsearch.action.ActionListenerImplementations$ResponseWrappingActionListener.onResponse(Ljava/lang/Object;)V (ActionListenerImplementations.java:247)
  at org.elasticsearch.action.ActionListenerImplementations$MappedActionListener.onResponse(Ljava/lang/Object;)V (ActionListenerImplementations.java:97)
  at org.elasticsearch.xpack.security.authc.AuthenticatorChain.authenticate(Lorg/elasticsearch/xpack/security/authc/Authenticator$Context;Lorg/elasticsearch/action/ActionListener;)V (AuthenticatorChain.java:93)
  at org.elasticsearch.xpack.security.authc.AuthenticationService.authenticate(Lorg/elasticsearch/xpack/security/authc/Authenticator$Context;Lorg/elasticsearch/action/ActionListener;)V (AuthenticationService.java:264)
  at org.elasticsearch.xpack.security.authc.AuthenticationService.authenticate(Ljava/lang/String;Lorg/elasticsearch/transport/TransportRequest;ZLorg/elasticsearch/action/ActionListener;)V (AuthenticationService.java:201)
  at org.elasticsearch.xpack.security.transport.ServerTransportFilter.authenticate(Ljava/lang/String;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/action/ActionListener;)V (ServerTransportFilter.java:127)
  at org.elasticsearch.xpack.security.transport.ServerTransportFilter.inbound(Ljava/lang/String;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/transport/TransportChannel;Lorg/elasticsearch/action/ActionListener;)V (ServerTransportFilter.java:105)
  at org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler.messageReceived(Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/transport/TransportChannel;Lorg/elasticsearch/tasks/Task;)V (SecurityServerTransportInterceptor.java:643)
  at org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/transport/TransportChannel;)V (RequestHandlerRegistry.java:76)
  at org.elasticsearch.transport.InboundHandler.doHandleRequest(Lorg/elasticsearch/transport/RequestHandlerRegistry;Lorg/elasticsearch/transport/TransportRequest;Lorg/elasticsearch/transport/TransportChannel;)V (InboundHandler.java:289)
  at org.elasticsearch.transport.InboundHandler$1.doRun()V (InboundHandler.java:302)
  at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun()V (ThreadContext.java:1044)
  at org.elasticsearch.common.util.concurrent.AbstractRunnable.run()V (AbstractRunnable.java:27)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1136)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:635)
  at java.lang.Thread.run()V (Thread.java:840)

Apparently the node remained in this state for long enough for the node to drop out and "never" rejoin.

This is another way that IndicesClusterStateService blocks on IO-like operations and harms the cluster stability as a result. See also:

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions