[ 
https://issues.apache.org/jira/browse/LUCENE-6482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Uwe Schindler updated LUCENE-6482:
----------------------------------
    Affects Version/s: 4.9.1
             Assignee: Uwe Schindler

> Class loading deadlock relating to NamedSPILoader
> -------------------------------------------------
>
>                 Key: LUCENE-6482
>                 URL: https://issues.apache.org/jira/browse/LUCENE-6482
>             Project: Lucene - Core
>          Issue Type: Bug
>    Affects Versions: 4.9.1
>            Reporter: Shikhar Bhushan
>            Assignee: Uwe Schindler
>
> This issue came up for us several times with Elasticsearch 1.3.4 (Lucene 
> 4.9.1), with many threads seeming deadlocked but RUNNABLE:
> {noformat}
> "elasticsearch[search77-es2][generic][T#43]" #160 daemon prio=5 os_prio=0 
> tid=0x00007f79180c5800 nid=0x3d1f in Object.wait() [0x00007f79d9289000]
>    java.lang.Thread.State: RUNNABLE
>       at org.apache.lucene.index.SegmentInfos.read(SegmentInfos.java:359)
>       at org.apache.lucene.index.SegmentInfos$1.doBody(SegmentInfos.java:457)
>       at 
> org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:912)
>       at 
> org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:758)
>       at org.apache.lucene.index.SegmentInfos.read(SegmentInfos.java:453)
>       at 
> org.elasticsearch.common.lucene.Lucene.readSegmentInfos(Lucene.java:98)
>       at org.elasticsearch.index.store.Store.readSegmentsInfo(Store.java:126)
>       at org.elasticsearch.index.store.Store.access$300(Store.java:76)
>       at 
> org.elasticsearch.index.store.Store$MetadataSnapshot.buildMetadata(Store.java:465)
>       at 
> org.elasticsearch.index.store.Store$MetadataSnapshot.<init>(Store.java:456)
>       at 
> org.elasticsearch.index.store.Store.readMetadataSnapshot(Store.java:281)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.listStoreMetaData(TransportNodesListShardStoreMetaData.java:186)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:140)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:61)
>       at 
> org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:277)
>       at 
> org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:268)
>       at 
> org.elasticsearch.transport.netty.MessageChannelHandler$RequestHandler.run(MessageChannelHandler.java:275)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> It didn't really make sense to see RUNNABLE threads in Object.wait(), but 
> this seems to be symptomatic of deadlocks in static initialization 
> (http://ternarysearch.blogspot.ru/2013/07/static-initialization-deadlock.html).
> I found LUCENE-5573 as an instance of this having come up with Lucene code 
> before.
> I'm not sure what exactly is going on, but the deadlock in this case seems to 
> involve these threads:
> {noformat}
> "elasticsearch[search77-es2][clusterService#updateTask][T#1]" #79 daemon 
> prio=5 os_prio=0 tid=0x00007f7b155ff800 nid=0xd49 in Object.wait() 
> [0x00007f79daed8000]
>    java.lang.Thread.State: RUNNABLE
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:408)
>       at java.lang.Class.newInstance(Class.java:433)
>       at org.apache.lucene.util.NamedSPILoader.reload(NamedSPILoader.java:67)
>       - locked <0x000000061fef4968> (a org.apache.lucene.util.NamedSPILoader)
>       at org.apache.lucene.util.NamedSPILoader.<init>(NamedSPILoader.java:47)
>       at org.apache.lucene.util.NamedSPILoader.<init>(NamedSPILoader.java:37)
>       at 
> org.apache.lucene.codecs.PostingsFormat.<clinit>(PostingsFormat.java:44)
>       at 
> org.elasticsearch.index.codec.postingsformat.PostingFormats.<clinit>(PostingFormats.java:67)
>       at 
> org.elasticsearch.index.codec.CodecModule.configurePostingsFormats(CodecModule.java:126)
>       at 
> org.elasticsearch.index.codec.CodecModule.configure(CodecModule.java:178)
>       at 
> org.elasticsearch.common.inject.AbstractModule.configure(AbstractModule.java:60)
>       - locked <0x000000061fef49e8> (a 
> org.elasticsearch.index.codec.CodecModule)
>       at 
> org.elasticsearch.common.inject.spi.Elements$RecordingBinder.install(Elements.java:204)
>       at 
> org.elasticsearch.common.inject.spi.Elements.getElements(Elements.java:85)
>       at 
> org.elasticsearch.common.inject.InjectorShell$Builder.build(InjectorShell.java:130)
>       at 
> org.elasticsearch.common.inject.InjectorBuilder.build(InjectorBuilder.java:99)
>       - locked <0x000000061fef4c10> (a 
> org.elasticsearch.common.inject.InheritingState)
>       at 
> org.elasticsearch.common.inject.InjectorImpl.createChildInjector(InjectorImpl.java:131)
>       at 
> org.elasticsearch.common.inject.ModulesBuilder.createChildInjector(ModulesBuilder.java:69)
>       at 
> org.elasticsearch.indices.InternalIndicesService.createIndex(InternalIndicesService.java:296)
>       - locked <0x000000061fef4cd0> (a 
> org.elasticsearch.indices.InternalIndicesService)
>       at 
> org.elasticsearch.indices.cluster.IndicesClusterStateService.applyNewIndices(IndicesClusterStateService.java:312)
>       at 
> org.elasticsearch.indices.cluster.IndicesClusterStateService.clusterChanged(IndicesClusterStateService.java:181)
>       - locked <0x000000061fef4e70> (a java.lang.Object)
>       at 
> org.elasticsearch.cluster.service.InternalClusterService$UpdateTask.run(InternalClusterService.java:444)
>       at 
> org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run(PrioritizedEsThreadPoolExecutor.java:153)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> {noformat}
> "elasticsearch[search77-es2][generic][T#1]" #80 daemon prio=5 os_prio=0 
> tid=0x00007f794400a000 nid=0xd4b in Object.wait() [0x00007f79dac56000]
>    java.lang.Thread.State: RUNNABLE
>       at 
> org.apache.lucene.codecs.simpletext.SimpleTextCodec.<init>(SimpleTextCodec.java:37)
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:408)
>       at java.lang.Class.newInstance(Class.java:433)
>       at org.apache.lucene.util.NamedSPILoader.reload(NamedSPILoader.java:67)
>       - locked <0x000000061fcf1f50> (a org.apache.lucene.util.NamedSPILoader)
>       at org.apache.lucene.util.NamedSPILoader.<init>(NamedSPILoader.java:47)
>       at org.apache.lucene.util.NamedSPILoader.<init>(NamedSPILoader.java:37)
>       at org.apache.lucene.codecs.Codec.<clinit>(Codec.java:41)
>       at org.apache.lucene.index.SegmentInfos.read(SegmentInfos.java:359)
>       at org.apache.lucene.index.SegmentInfos$1.doBody(SegmentInfos.java:457)
>       at 
> org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:912)
>       at 
> org.apache.lucene.index.SegmentInfos$FindSegmentsFile.run(SegmentInfos.java:758)
>       at org.apache.lucene.index.SegmentInfos.read(SegmentInfos.java:453)
>       at 
> org.elasticsearch.common.lucene.Lucene.readSegmentInfos(Lucene.java:98)
>       at org.elasticsearch.index.store.Store.readSegmentsInfo(Store.java:126)
>       at org.elasticsearch.index.store.Store.access$300(Store.java:76)
>       at 
> org.elasticsearch.index.store.Store$MetadataSnapshot.buildMetadata(Store.java:465)
>       at 
> org.elasticsearch.index.store.Store$MetadataSnapshot.<init>(Store.java:456)
>       at 
> org.elasticsearch.index.store.Store.readMetadataSnapshot(Store.java:281)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.listStoreMetaData(TransportNodesListShardStoreMetaData.java:186)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:140)
>       at 
> org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData.nodeOperation(TransportNodesListShardStoreMetaData.java:61)
>       at 
> org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:277)
>       at 
> org.elasticsearch.action.support.nodes.TransportNodesOperationAction$NodeTransportHandler.messageReceived(TransportNodesOperationAction.java:268)
>       at 
> org.elasticsearch.transport.netty.MessageChannelHandler$RequestHandler.run(MessageChannelHandler.java:275)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Full thread dump: https://gist.github.com/shikhar/d0f6d2d008f45d2d4f91



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org

Reply via email to