You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by "Jessica Cheng Mallet (JIRA)" <ji...@apache.org> on 2015/11/20 20:48:11 UTC

[jira] [Created] (SOLR-8327) SolrDispatchFilter is not caching new state format, which results in live fetch from ZK per request if node does not contain core from collection

Jessica Cheng Mallet created SOLR-8327:
------------------------------------------

             Summary: SolrDispatchFilter is not caching new state format, which results in live fetch from ZK per request if node does not contain core from collection
                 Key: SOLR-8327
                 URL: https://issues.apache.org/jira/browse/SOLR-8327
             Project: Solr
          Issue Type: Bug
          Components: SolrCloud
    Affects Versions: 5.3
            Reporter: Jessica Cheng Mallet


While perf testing with non-solrj client (request can be sent to any solr node), we noticed a huge amount of data from Zookeeper in our tcpdump (~1G for 20 second dump). From the thread dump, we noticed this:

java.lang.Object.wait (Native Method)
java.lang.Object.wait (Object.java:503)
org.apache.zookeeper.ClientCnxn.submitRequest (ClientCnxn.java:1309)
org.apache.zookeeper.ZooKeeper.getData (ZooKeeper.java:1152)
org.apache.solr.common.cloud.SolrZkClient$7.execute (SolrZkClient.java:345)
org.apache.solr.common.cloud.SolrZkClient$7.execute (SolrZkClient.java:342)
org.apache.solr.common.cloud.ZkCmdExecutor.retryOperation (ZkCmdExecutor.java:61)
org.apache.solr.common.cloud.SolrZkClient.getData (SolrZkClient.java:342)
org.apache.solr.common.cloud.ZkStateReader.getCollectionLive (ZkStateReader.java:841)
org.apache.solr.common.cloud.ZkStateReader$7.get (ZkStateReader.java:515)
org.apache.solr.common.cloud.ClusterState.getCollectionOrNull (ClusterState.java:175)
org.apache.solr.common.cloud.ClusterState.getLeader (ClusterState.java:98)
org.apache.solr.servlet.HttpSolrCall.getCoreByCollection (HttpSolrCall.java:784)
org.apache.solr.servlet.HttpSolrCall.init (HttpSolrCall.java:272)
org.apache.solr.servlet.HttpSolrCall.call (HttpSolrCall.java:417)
org.apache.solr.servlet.SolrDispatchFilter.doFilter (SolrDispatchFilter.java:210)
org.apache.solr.servlet.SolrDispatchFilter.doFilter (SolrDispatchFilter.java:179)
org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter (ServletHandler.java:1652)
org.eclipse.jetty.servlet.ServletHandler.doHandle (ServletHandler.java:585)
org.eclipse.jetty.server.handler.ScopedHandler.handle (ScopedHandler.java:143)
org.eclipse.jetty.security.SecurityHandler.handle (SecurityHandler.java:577)
org.eclipse.jetty.server.session.SessionHandler.doHandle (SessionHandler.java:223)
org.eclipse.jetty.server.handler.ContextHandler.doHandle (ContextHandler.java:1127)
org.eclipse.jetty.servlet.ServletHandler.doScope (ServletHandler.java:515)
org.eclipse.jetty.server.session.SessionHandler.doScope (SessionHandler.java:185)
org.eclipse.jetty.server.handler.ContextHandler.doScope (ContextHandler.java:1061)
org.eclipse.jetty.server.handler.ScopedHandler.handle (ScopedHandler.java:141)
org.eclipse.jetty.server.handler.ContextHandlerCollection.handle (ContextHandlerCollection.java:215)
org.eclipse.jetty.server.handler.HandlerCollection.handle (HandlerCollection.java:110)
org.eclipse.jetty.server.handler.HandlerWrapper.handle (HandlerWrapper.java:97)
org.eclipse.jetty.server.Server.handle (Server.java:499)
org.eclipse.jetty.server.HttpChannel.handle (HttpChannel.java:310)
org.eclipse.jetty.server.HttpConnection.onFillable (HttpConnection.java:257)
org.eclipse.jetty.io.AbstractConnection$2.run (AbstractConnection.java:540)
org.eclipse.jetty.util.thread.QueuedThreadPool.runJob (QueuedThreadPool.java:635)
org.eclipse.jetty.util.thread.QueuedThreadPool$3.run (QueuedThreadPool.java:555)
java.lang.Thread.run (Thread.java:745)

Looks like SolrDispatchFilter doesn't have caching similar to the collectionStateCache in CloudSolrClient, so if the node doesn't know about a collection in the new state format, it just live-fetch it from Zookeeper on every request.



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

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