You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-user@lucene.apache.org by Damien Kamerman <da...@gmail.com> on 2015/10/19 06:59:28 UTC
SOLR-7191 SolrCloud 5 with thousands of collections
Hi All,
I've had a first look at porting the patch I did for SOLR-7191 (SolrCloud
with thousands of collections) in Solr 4.10 to the Solr trunk (1708905).
Now I created 6,000 collections (3 nodes; 2 x replicas) and re-started the
3 nodes. What I noticed is that the cloud is starting but slowly. All
the org.apache.solr.core.CoreContainer.create()
threads are blocked in the ZkStateReader. I was hoping the changes to
clusterstate.json from global to per collection would reduce the
contention. Comments appreciated.
example jstacks:
"coreLoadExecutor-6-thread-24-processing-n:ftet1:8003_solr" #70 prio=5
os_prio=64 tid=0x0000000000bcd800 nid=0x88 waiting for monitor entry
[0x00007fefb29bc000]
java.lang.Thread.State: BLOCKED (on object monitor)
at
org.apache.solr.common.cloud.ZkStateReader.addCollectionWatch(ZkStateReader.java:1048)
- waiting to lock <0x00007ff0403ff020> (a
org.apache.solr.common.cloud.ZkStateReader)
at
org.apache.solr.cloud.ZkController.preRegister(ZkController.java:1561)
at org.apache.solr.core.CoreContainer.create(CoreContainer.java:726)
at org.apache.solr.core.CoreContainer$1.call(CoreContainer.java:451)
at org.apache.solr.core.CoreContainer$1.call(CoreContainer.java:442)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
org.apache.solr.common.util.ExecutorUtil$MDCAwareThreadPoolExecutor$1.run(ExecutorUtil.java:231)
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)
"zkCallback-4-thread-80-processing-n:ftet1:8003_solr" #268 prio=5
os_prio=64 tid=0x0000000002ee0000 nid=0x134 in Object.wait()
[0x00007fefaed2d000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:502)
at
org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1342)
- locked <0x00007ff0be17e600> (a
org.apache.zookeeper.ClientCnxn$Packet)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1153)
at
org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:353)
at
org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:350)
at
org.apache.solr.common.cloud.ZkCmdExecutor.retryOperation(ZkCmdExecutor.java:61)
at
org.apache.solr.common.cloud.SolrZkClient.getData(SolrZkClient.java:350)
at
org.apache.solr.common.cloud.ZkStateReader.fetchCollectionState(ZkStateReader.java:1030)
at
org.apache.solr.common.cloud.ZkStateReader.getCollectionLive(ZkStateReader.java:1015)
at
org.apache.solr.common.cloud.ZkStateReader$LazyCollectionRef.get(ZkStateReader.java:550)
at
org.apache.solr.common.cloud.ClusterState.getCollections(ClusterState.java:207)
at
org.apache.solr.common.cloud.ZkStateReader.constructState(ZkStateReader.java:462)
at
org.apache.solr.common.cloud.ZkStateReader.access$600(ZkStateReader.java:57)
at
org.apache.solr.common.cloud.ZkStateReader$StateWatcher.process(ZkStateReader.java:864)
- locked <0x00007ff0403ff020> (a
org.apache.solr.common.cloud.ZkStateReader)
at
org.apache.solr.common.cloud.SolrZkClient$3$1.run(SolrZkClient.java:269)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
org.apache.solr.common.util.ExecutorUtil$MDCAwareThreadPoolExecutor$1.run(ExecutorUtil.java:231)
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)
Re: SOLR-7191 SolrCloud 5 with thousands of collections
Posted by Damien Kamerman <da...@gmail.com>.
OK, turned out ZkStateReader.constructState() was only calling
ClusterState.getCollections()
for log.debug(). I removed that and the next bottleneck is talking
to ZkStateReader.fetchCollectionState.
"coreZkRegister-4-thread-14-processing-n:ftet1:8003_solr
x:t_1558_shard1_replica1 s:shard1 c:t_1558 r:core_node1" #151 prio=5
os_prio=64 tid=0x0000000005568800 nid=0xc8 in Object.wait()
[0x00007fefb117c000]
java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:502)
at
org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1342)
- locked <0x00007fff50fadf70> (a
org.apache.zookeeper.ClientCnxn$Packet)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1153)
at
org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:353)
at
org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:350)
at
org.apache.solr.common.cloud.ZkCmdExecutor.retryOperation(ZkCmdExecutor.java:61)
at
org.apache.solr.common.cloud.SolrZkClient.getData(SolrZkClient.java:350)
at
org.apache.solr.common.cloud.ZkStateReader.fetchCollectionState(ZkStateReader.java:1029)
at
org.apache.solr.common.cloud.ZkStateReader.updateClusterState(ZkStateReader.java:260)
- locked <0x00007ff040b92270> (a
org.apache.solr.common.cloud.ZkStateReader)
at
org.apache.solr.cloud.ZkController.register(ZkController.java:979)
at
org.apache.solr.cloud.ZkController.register(ZkController.java:881)
at org.apache.solr.core.ZkContainer$2.run(ZkContainer.java:184)
at
org.apache.solr.common.util.ExecutorUtil$MDCAwareThreadPoolExecutor$1.run(ExecutorUtil.java:231)
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)
On 19 October 2015 at 15:59, Damien Kamerman <da...@gmail.com> wrote:
> Hi All,
>
> I've had a first look at porting the patch I did for SOLR-7191 (SolrCloud
> with thousands of collections) in Solr 4.10 to the Solr trunk (1708905).
> Now I created 6,000 collections (3 nodes; 2 x replicas) and re-started the
> 3 nodes. What I noticed is that the cloud is starting but slowly. All the org.apache.solr.core.CoreContainer.create()
> threads are blocked in the ZkStateReader. I was hoping the changes to
> clusterstate.json from global to per collection would reduce the
> contention. Comments appreciated.
>
> example jstacks:
> "coreLoadExecutor-6-thread-24-processing-n:ftet1:8003_solr" #70 prio=5
> os_prio=64 tid=0x0000000000bcd800 nid=0x88 waiting for monitor entry
> [0x00007fefb29bc000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at
> org.apache.solr.common.cloud.ZkStateReader.addCollectionWatch(ZkStateReader.java:1048)
> - waiting to lock <0x00007ff0403ff020> (a
> org.apache.solr.common.cloud.ZkStateReader)
> at
> org.apache.solr.cloud.ZkController.preRegister(ZkController.java:1561)
> at
> org.apache.solr.core.CoreContainer.create(CoreContainer.java:726)
> at
> org.apache.solr.core.CoreContainer$1.call(CoreContainer.java:451)
> at
> org.apache.solr.core.CoreContainer$1.call(CoreContainer.java:442)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> org.apache.solr.common.util.ExecutorUtil$MDCAwareThreadPoolExecutor$1.run(ExecutorUtil.java:231)
> 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)
>
> "zkCallback-4-thread-80-processing-n:ftet1:8003_solr" #268 prio=5
> os_prio=64 tid=0x0000000002ee0000 nid=0x134 in Object.wait()
> [0x00007fefaed2d000]
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> at java.lang.Object.wait(Object.java:502)
> at
> org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1342)
> - locked <0x00007ff0be17e600> (a
> org.apache.zookeeper.ClientCnxn$Packet)
> at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1153)
> at
> org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:353)
> at
> org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:350)
> at
> org.apache.solr.common.cloud.ZkCmdExecutor.retryOperation(ZkCmdExecutor.java:61)
> at
> org.apache.solr.common.cloud.SolrZkClient.getData(SolrZkClient.java:350)
> at
> org.apache.solr.common.cloud.ZkStateReader.fetchCollectionState(ZkStateReader.java:1030)
> at
> org.apache.solr.common.cloud.ZkStateReader.getCollectionLive(ZkStateReader.java:1015)
> at
> org.apache.solr.common.cloud.ZkStateReader$LazyCollectionRef.get(ZkStateReader.java:550)
> at
> org.apache.solr.common.cloud.ClusterState.getCollections(ClusterState.java:207)
> at
> org.apache.solr.common.cloud.ZkStateReader.constructState(ZkStateReader.java:462)
> at
> org.apache.solr.common.cloud.ZkStateReader.access$600(ZkStateReader.java:57)
> at
> org.apache.solr.common.cloud.ZkStateReader$StateWatcher.process(ZkStateReader.java:864)
> - locked <0x00007ff0403ff020> (a
> org.apache.solr.common.cloud.ZkStateReader)
> at
> org.apache.solr.common.cloud.SolrZkClient$3$1.run(SolrZkClient.java:269)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> org.apache.solr.common.util.ExecutorUtil$MDCAwareThreadPoolExecutor$1.run(ExecutorUtil.java:231)
> 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)
>
--
Damien Kamerman