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 Vijay Sekhri <se...@gmail.com> on 2015/01/25 22:02:05 UTC

replicas goes in recovery mode right after update

We have a cluster of solr cloud server with 10 shards and 4 replicas in
each shard in our stress environment. In our prod environment we will have
10 shards and 15 replicas in each shard. Our current commit settings are as
follows

*    <autoSoftCommit>*
*        <maxDocs>500000</maxDocs>*
*        <maxTime>180000</maxTime>*
*    </autoSoftCommit>*
*    <autoCommit>*
*        <maxDocs>2000000</maxDocs>*
*        <maxTime>180000</maxTime>*
*        <openSearcher>false</openSearcher>*
*    </autoCommit>*


We indexed roughly 90 Million docs. We have two different ways to index
documents a) Full indexing. It takes 4 hours to index 90 Million docs and
the rate of docs coming to the searcher is around 6000 per second b)
Incremental indexing. It takes an hour to indexed delta changes. Roughly
there are 3 million changes and rate of docs coming to the searchers is 2500
per second

We have two collections search1 and search2. When we do full indexing , we
do it in search2 collection while search1 is serving live traffic. After it
finishes we swap the collection using aliases so that the search2
collection serves live traffic while search1 becomes available for next
full indexing run. When we do incremental indexing we do it in the search1
collection which is serving live traffic.

All our searchers have 12 GB of RAM available and have quad core Intel(R)
Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
jboss and solr in it . All 12 GB is available as heap for the java
process.  We have observed that the heap memory of the java process average
around 8 - 10 GB. All searchers have final index size of 9 GB. So in total
there are 9X10 (shards) =  90GB worth of index files.

 We have observed the following issue when we trigger indexing . In about
10 minutes after we trigger indexing on 14 parallel hosts, the replicas
goes in to recovery mode. This happens to all the shards . In about 20
minutes more and more replicas start going into recovery mode. After about
half an hour all replicas except the leader are in recovery mode. We cannot
throttle the indexing load as that will increase our overall indexing time.
So to overcome this issue, we remove all the replicas before we trigger the
indexing and then add them back after the indexing finishes.

We observe the same behavior of replicas going into recovery when we do
incremental indexing. We cannot remove replicas during our incremental
indexing because it is also serving live traffic. We tried to throttle our
indexing speed , however the cluster still goes into recovery .

If we leave the cluster as it , when the indexing finishes , it eventually
recovers after a while. As it is serving live traffic we cannot have these
replicas go into recovery mode because it degrades the search performance
also , our tests have shown.

We have tried different commit settings like below

a) No auto soft commit, no auto hard commit and a commit triggered at the
end of indexing b) No auto soft commit, yes auto hard commit and a commit
in the end of indexing
c) Yes auto soft commit , no auto hard commit
d) Yes auto soft commit , yes auto hard commit
e) Different frequency setting for commits for above. Please NOTE that we
have tried 15 minute soft commit setting and 30 minutes hard commit
settings. Same time settings for both, 30 minute soft commit and an hour
hard commit setting

Unfortunately all the above yields the same behavior . The replicas still
goes in recovery We have increased the zookeeper timeout from 30 seconds to
5 minutes and the problem persists. Is there any setting that would fix
this issue ?

-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Shawn, Erick
>From another replicas right after the same error it seems the leader
initiates the recovery of the replicas. This one has a bit different log
information than the other one that went into recovery. I am not sure if
this helps in diagnosing

Caused by: java.io.IOException: JBWEB002020: Invalid chunk header
        at
org.apache.coyote.http11.filters.ChunkedInputFilter.parseChunkHeader(ChunkedInputFilter.java:281)
        at
org.apache.coyote.http11.filters.ChunkedInputFilter.doRead(ChunkedInputFilter.java:134)
        at
org.apache.coyote.http11.InternalInputBuffer.doRead(InternalInputBuffer.java:697)
        at org.apache.coyote.Request.doRead(Request.java:438)
        at
org.apache.catalina.connector.InputBuffer.realReadBytes(InputBuffer.java:341)
        ... 31 more

21:55:07,678 INFO  [org.apache.solr.handler.admin.CoreAdminHandler]
(http-/10.235.43.57:8680-32) It has been requested that we recover:
core=search1_shard4_replica13
21:55:07,678 INFO  [org.apache.solr.servlet.SolrDispatchFilter]
(http-/10.235.43.57:8680-32) [admin] webapp=null path=/admin/cores
params={action=REQUESTRECOVERY&core=search1_shard4_replica13&wt=javabin&version=2}
status=0 QTime=0
21:55:07,678 INFO  [org.apache.solr.cloud.ZkController] (Thread-443)
publishing core=search1_shard4_replica13 state=recovering collection=search1
21:55:07,678 INFO  [org.apache.solr.cloud.ZkController] (Thread-443)
numShards not found on descriptor - reading it from system property
21:55:07,681 INFO  [org.apache.solr.cloud.ZkController] (Thread-443) Wrote
recovering to /collections/search1/leader_initiated_recovery
/shard4/core_node192


On Mon, Jan 26, 2015 at 10:34 PM, Vijay Sekhri <se...@gmail.com>
wrote:

> Hi Shawn, Erick
> So it turned out that once we increased our indexing rate to the original
> full indexing rate  the replicas went back into recovery no matter what the
> zk timeout setting was. Initially we though that increasing the timeout is
> helping but apparently not . We just decreased indexing rate and that
> caused less replicas to go in recovery. Once we have our full indexing rate
> almost all replicas went into recovery no matter what the zk timeout or the
> ticktime setting were. We reverted back the ticktime to original 2 seconds
>
> So we investigated further and after checking the logs we found this
> exception happening right before the recovery process is initiated. We
> observed this on two different replicas that went into recovery. We are not
> sure if this is a coincidence or a real problem . Notice we were also
> putting some search query load while indexing to trigger the recovery
> behavior
>
> 22:00:32,493 INFO  [org.apache.solr.cloud.RecoveryStrategy]
> (rRecoveryThread) Finished recovery process. core=search1_shard5_replica2
> 22:00:32,503 INFO  [org.apache.solr.common.cloud.ZkStateReader]
> (zkCallback-2-thread-66) A cluster state change: WatchedEvent
> state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
> occurred - updating... (live nodes size: 22)
> 22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: trigger
> flush: activeBytes=101796784 deleteBytes=3061644 vs limit=104857600
> 22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: thread
> state has 12530488 bytes; docInRAM=2051
> 22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: thread
> state has 12984633 bytes; docInRAM=2205
>
>
> 22:00:40,861 ERROR [org.apache.solr.core.SolrCore] (http-/10.235.46.36:8580-32)
> ClientAbortException: * java.io.IOException: JBWEB002020: Invalid chunk
> header*
>         at
> org.apache.catalina.connector.InputBuffer.realReadBytes(InputBuffer.java:351)
>         at
> org.apache.tomcat.util.buf.ByteChunk.substract(ByteChunk.java:422)
>         at
> org.apache.catalina.connector.InputBuffer.read(InputBuffer.java:373)
>         at
> org.apache.catalina.connector.CoyoteInputStream.read(CoyoteInputStream.java:193)
>         at
> org.apache.solr.common.util.FastInputStream.readWrappedStream(FastInputStream.java:80)
>         at
> org.apache.solr.common.util.FastInputStream.refill(FastInputStream.java:89)
>         at
> org.apache.solr.common.util.FastInputStream.readByte(FastInputStream.java:192)
>         at
> org.apache.solr.common.util.JavaBinCodec.unmarshal(JavaBinCodec.java:111)
>         at
> org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec.unmarshal(JavaBinUpdateRequestCodec.java:173)
>         at
> org.apache.solr.handler.loader.JavabinLoader.parseAndLoadDocs(JavabinLoader.java:106)
>         at
> org.apache.solr.handler.loader.JavabinLoader.load(JavabinLoader.java:58)
>         at
> org.apache.solr.handler.UpdateRequestHandler$1.load(UpdateRequestHandler.java:99)
>         at
> org.apache.solr.handler.ContentStreamHandlerBase.handleRequestBody(ContentStreamHandlerBase.java:74)
>         at
> org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
>         at org.apache.solr.core.SolrCore.execute(SolrCore.java:1967)
>         at
> org.apache.solr.servlet.SolrDispatchFilter.execute(SolrDispatchFilter.java:777)
>         at
> org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:418)
>         at
> org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:207)
>         at
> org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:246)
>         at
> org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:214)
>         at
> org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:230)
>         at
> org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:149)
>         at
> org.jboss.as.web.security.SecurityContextAssociationValve.invoke(SecurityContextAssociationValve.java:169)
>         at
> org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:145)
>         at
> org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:97)
>         at
> org.apache.catalina.valves.AccessLogValve.invoke(AccessLogValve.java:559)
>         at
> org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:102)
>         at
> org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:336)
>         at
> org.apache.coyote.http11.Http11Processor.process(Http11Processor.java:856)
>                                           at
> org.apache.tomcat.util.net.JIoEndpoint$Worker.run(JIoEndpoint.java:920)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: java.io.IOException: JBWEB002020: Invalid chunk header
>         at
> org.apache.coyote.http11.filters.ChunkedInputFilter.parseChunkHeader(ChunkedInputFilter.java:281)
>         at
> org.apache.coyote.http11.filters.ChunkedInputFilter.doRead(ChunkedInputFilter.java:134)
>         at
> org.apache.coyote.http11.InternalInputBuffer.doRead(InternalInputBuffer.java:697)
>         at org.apache.coyote.Request.doRead(Request.java:438)
>         at
> org.apache.catalina.connector.InputBuffer.realReadBytes(InputBuffer.java:341)
>         ... 31 more
> 22:00:40,864 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: new
> segment has 0 deleted docs
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: new
> segment has no vectors; no norms; no docValues; prox; freqs
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]:
> flushedFiles=[_227_Lucene41_0.tip, _227.fdx, _227_Lucene41_0.tim, _227.fdt,
> _227_Lucene41_0.doc, _227.fnm, _227_Lucene41_0.pos]
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: flushed
> codec=Lucene410
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: flushed:
> segment=_227 ramUsed=12.383 MB newFlushedSize(includes docstores)=4.444 MB
> docs/MB=496.165
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [DW][http-/10.235.46.36:8580-27]:
> publishFlushedSegment seg-private updates=null
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [IW][http-/10.235.46.36:8580-27]:
> publishFlushedSegment
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [BD][http-/10.235.46.36:8580-27]: push
> deletes  16553 deleted terms (unique count=16553) bytesUsed=222240
> delGen=17 packetCount=1 totBytesUsed=222240
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [IW][http-/10.235.46.36:8580-27]: publish
> sets newSegment delGen=18 seg=_227(4.10.0):C2205
> 22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [IFD][http-/10.235.46.36:8580-27]: now
> checkpoint "_12l(4.10.0):C676510/354682:delGen=208
> _1wo(4.10.0):C420813/69819:delGen=40 _1bs(4.10.0):C229862/110117:delGen=77
> _1ga(4.10.0):C322386/84434:delGen=91 _1nc(4.10.0):C209477/2949:delGen=52
> _223(4.10.0):C206753/11:delGen=1 _1ug(4.10.0):C7185/1:delGen=1
> _20n(4.10.0):C59384/2287:delGen=7 _1zm(4.10.0):C78927/1581:delGen=12
> _20v(4.10.0):C4317 _20u(4.10.0):C5243/1:delGen=1 _20z(4.10.0):C3288
> _20y(4.10.0):C4614/1:delGen=1 _21i(4.10.0):C2190 _21f(4.10.0):C16
> _21g(4.10.0):C67 _21h(4.10.0):C18 _21o(4.10.0):C116
> _21q(4.10.0):C1243/1:delGen=1 _21r(4.10.0):C39 _21s(4.10.0):C92
> _21t(4.10.0):C1 _21u(4.10.0):C1 _222(4.10.0):C26988
> _224(4.10.0):C28193/1:delGen=1 _225(4.10.0):C25966 _227(4.10.0):C2205" [27
> segments ; isCommit = false]
> 22:00:40,866 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [IFD][http-/10.235.46.36:8580-27]: 0 msec to
> checkpoint
> 22:00:40,866 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [TMP][http-/10.235.46.36:8580-27]:
> findMerges: 27 segments
> 10.235.46.36:8580-27]: now merge
> 22:00:40,867 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [CMS][http-/10.235.46.36:8580-27]:   index:
> _12l(4.10.0):C676510/354682:delGen=208 _1wo(4.10.0):C420813/69819:delGen=40
> _1bs(4.10.0):C229862/110117:delGen=77 _1ga(4.10.0):C322386/84434:delGen=91
> _1nc(4.10.0):C209477/2949:delGen=52 _223(4.10.0):C206753/11:delGen=1
> _1ug(4.10.0):C7185/1:delGen=1 _20n(4.10.0):C59384/2287:delGen=7
> _1zm(4.10.0):C78927/1581:delGen=12 _20v(4.10.0):C4317
> _20u(4.10.0):C5243/1:delGen=1 _20z(4.10.0):C3288
> _20y(4.10.0):C4614/1:delGen=1 _21i(4.10.0):C2190 _21f(4.10.0):C16
> _21g(4.10.0):C67 _21h(4.10.0):C18 _21o(4.10.0):C116
> _21q(4.10.0):C1243/1:delGen=1 _21r(4.10.0):C39 _21s(4.10.0):C92
> _21t(4.10.0):C1 _21u(4.10.0):C1 _222(4.10.0):C26988
> _224(4.10.0):C28193/1:delGen=1 _225(4.10.0):C25966 _227(4.10.0):C2205
> 22:00:40,867 INFO  [org.apache.solr.update.LoggingInfoStream]
> (http-/10.235.46.36:8580-27) [CMS][http-/10.235.46.36:8580-27]:   no more
> merges pending; now return
> 22:00:41,885 INFO  [org.apache.solr.handler.admin.CoreAdminHandler]
> (http-/10.235.46.36:8580-32) It has been requested that we recover:
> core=search1_shard5_replica2
> 22:00:41,886 INFO  [org.apache.solr.servlet.SolrDispatchFilter]
> (http-/10.235.46.36:8580-32) [admin] webapp=null path=/admin/cores
> params={action=REQUESTRECOVERY&core=search1_shard5_replica2&wt=javabin&version=2}
> status=0 QTime=1
> 22:00:41,886 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)
> publishing core=search1_shard5_replica2 state=recovering collection=search1
> 22:00:41,886 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)
> numShards not found on descriptor - reading it from system property
>
> 22:00:41,889 INFO  [org.apache.solr.common.cloud.ZkStateReader]
> (zkCallback-2-thread-66) A cluster state change: WatchedEvent
> state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
> occurred - updating... (live nodes size: 22)
> 22:00:41,890 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)*
> Wrote recovering to
> /collections/search1/leader_initiated_recovery/shard5/core_node223*
> 22:00:41,892 INFO  [org.apache.solr.update.DefaultSolrCoreState]
> (Thread-472) Running recovery - first canceling any ongoing recovery
> 22:00:41,893 INFO  [org.apache.solr.cloud.RecoveryStrategy]
> (RecoveryThread) Starting recovery process.  core=search1_shard5_replica2
> recoveringAfterStartup=false
> 22:00:41,894 INFO  [org.apache.solr.cloud.RecoveryStrategy]
> (RecoveryThread) Publishing state of core search1_shard5_replica2 as
> recovering, leader is http://XXXXXXXXXXCHANGEDONPURPOSEXXXX:8680/solr/search1/
> and I am http://solrx341p.qa.ch3.s.com:8580/solr/search1_shard5_replica2/
> 22:00:41,894 INFO  [org.apache.solr.cloud.ZkController] (RecoveryThread)
> publishing core=search1_shard5_replica2 state=recovering collection=search1
> 22:00:41,894 INFO  [org.apache.solr.cloud.ZkController] (RecoveryThread)
> numShards not found on descriptor - reading it from system property
> 22:00:41,896 INFO  [org.apache.solr.cloud.RecoveryStrategy]
> (RecoveryThread) Sending prep recovery command to
> http://XXXXXXXXXCHANGEDONPURPOSEXXXXXXXX:8680/solr; WaitForState:
> action=PREPRECOVERY&core=search1&nodeName=XXXXXXXXXXXXXXCHANGEDONPURPOSE
> XXXXXXXXXXXXXXX%3A8580_solr&coreNodeName=core_node223&state=recovering&checkLive=true&onlyIfLeader=true&onlyIfLeaderActive=true
> 22:00:42,006 INFO  [org.apache.solr.common.cloud.ZkStateReader]
> (zkCallback-2-thread-66) A cluster state change: WatchedEvent
> state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
> occurred - updating... (live nodes size: 22)
>
>
>
>
> 15Gb of heap is allocated to these searchers and it hardly goes over 8GB.
> Full GC does not happen. We write gc logs to separate file and also
> monitored these process via visualvm . It seems to have enough memory
>
>
> On Mon, Jan 26, 2015 at 5:13 PM, Shawn Heisey <ap...@elyograg.org> wrote:
>
>> On 1/26/2015 2:26 PM, Vijay Sekhri wrote:
>> > Hi Erick,
>> > In solr.xml file I had zk timeout set to/ <int
>> > name="zkClientTimeout">${zkClientTimeout:450000}</int>/
>> > One thing that made a it a bit better now is the zk tick time and
>> > syncLimit settings. I set it to a higher value as below. This may not
>> > be advisable though.
>> >
>> > tickTime=30000
>> > initLimit=30
>> > syncLimit=20
>> >
>> > Now we observed that replicas do not go in recovery that often as
>> > before. In the whole cluster at a given time I would have a couple of
>> > replicas in recovery whereas earlier it were multiple replicas from
>> > every shard .
>> > On the wiki https://wiki.apache.org/solr/SolrCloudit says the "The
>> > maximum is 20 times the tickTime." in the FAQ so I decided to increase
>> > the tick time. Is this the correct approach ?
>>
>> The default zkClientTimeout on recent Solr versions is 30 seconds, up
>> from 15 in slightly older releases.
>>
>> Those values of 15 or 30 seconds are a REALLY long time in computer
>> terms, and if you are exceeding that timeout on a regular basis,
>> something is VERY wrong with your Solr install.  Rather than take steps
>> to increase your timeout beyond the normal maximum of 40 seconds (20
>> times a tickTime of 2 seconds), figure out why you're exceeding that
>> timeout and fix the performance problem.  The zkClientTimeout value that
>> you have set, 450 seconds, is seven and a half *MINUTES*.  Nothing in
>> Solr should ever take that long.
>>
>> "Not enough memory in the server" is by far the most common culprit for
>> performance issues.  Garbage collection pauses are a close second.
>>
>> I don't actually know this next part for sure, because I've never looked
>> into the code, but I believe that increasing the tickTime, especially to
>> a value 15 times higher than default, might make all zookeeper
>> operations a lot slower.
>>
>> Thanks,
>> Shawn
>>
>>
>
>
> --
> *********************************************
> Vijay Sekhri
> *********************************************
>



-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Erick,

@ichattopadhyaya  beat me to it already yesterday. So we are good
-cheers
Vijay

On Wed, Jan 28, 2015 at 1:30 PM, Erick Erickson <er...@gmail.com>
wrote:

> Vijay:
>
> Thanks for reporting this back!  Could I ask you to post a new patch with
> your correction? Please use the same patch name
> (SOLR-5850.patch), and include a note about what you found (I've already
> added a comment).
>
> Thanks!
> Erick
>
> On Wed, Jan 28, 2015 at 9:18 AM, Vijay Sekhri <se...@gmail.com>
> wrote:
>
> > Hi Shawn,
> > Thank you so much for the assistance. Building is not a problem . Back in
> > the days I have worked with linking, compiling and  building C , C++
> > software . Java is a piece of cake.
> > We have built the new war from the source version 4.10.3 and our
> > preliminary tests have shown that our issue (replicas in recovery on high
> > load)* is resolved *. We will continue to do more testing and confirm .
> > Please note that the *patch is BUGGY*.
> >
> > It removed the break statement within while loop because of which,
> whenever
> > we send a list of docs it would hang (API CloudSolrServer.add) , but it
> > would work if send one doc at a time.
> >
> > It took a while to figure out why that is happening. Once we put the
> break
> > statement back it worked like a charm.
> > Furthermore the patch has
> >
> >
> solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
> > which should be
> >
> >
> solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java
> >
> > Finally checking if(!offer) is sufficient than using if(offer == false)
> > Last but not the least having a configurable queue size and timeouts
> > (managed via solrconfig) would be quite helpful
> > Thank you once again for your help.
> >
> > Vijay
> >
> > On Tue, Jan 27, 2015 at 6:20 PM, Shawn Heisey <ap...@elyograg.org>
> wrote:
> >
> > > On 1/27/2015 2:52 PM, Vijay Sekhri wrote:
> > > > Hi Shawn,
> > > > Here is some update. We found the main issue
> > > > We have configured our cluster to run under jetty and when we tried
> > full
> > > > indexing, we did not see the original Invalid Chunk error. However
> the
> > > > replicas still went into recovery
> > > > All this time we been trying to look into replicas logs to diagnose
> the
> > > > issue. The problem seem to be at the leader side. When we looked into
> > > > leader logs, we found the following on all the leaders
> > > >
> > > > 3439873 [qtp1314570047-92] WARN
> > > >  org.apache.solr.update.processor.DistributedUpdateProcessor  – Error
> > > > sending update
> > > > *java.lang.IllegalStateException: Queue full*
> > >
> > > <snip>
> > >
> > > > There is a similar bug reported around this
> > > > https://issues.apache.org/jira/browse/SOLR-5850
> > > >
> > > > and it seem to be in OPEN status. Is there a way we can configure the
> > > queue
> > > > size and increase it ? or is there a version of solr that has this
> > issue
> > > > resolved already?
> > > > Can you suggest where we go from here to resolve this ? We can
> repatch
> > > the
> > > > war file if that is what you would recommend .
> > > > In the end our initial speculation about solr unable to handle so
> many
> > > > update is correct. We do not see this issue when the update load is
> > less.
> > >
> > > Are you in a position where you can try the patch attached to
> > > SOLR-5850?  You would need to get the source code for the version
> you're
> > > on (or perhaps a newer 4.x version), patch it, and build Solr yourself.
> > > If you have no experience building java packages from source, this
> might
> > > prove to be difficult.
> > >
> > > Thanks,
> > > Shawn
> > >
> > >
> >
> >
> > --
> > *********************************************
> > Vijay Sekhri
> > *********************************************
> >
>



-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Erick Erickson <er...@gmail.com>.
Vijay:

Thanks for reporting this back!  Could I ask you to post a new patch with
your correction? Please use the same patch name
(SOLR-5850.patch), and include a note about what you found (I've already
added a comment).

Thanks!
Erick

On Wed, Jan 28, 2015 at 9:18 AM, Vijay Sekhri <se...@gmail.com> wrote:

> Hi Shawn,
> Thank you so much for the assistance. Building is not a problem . Back in
> the days I have worked with linking, compiling and  building C , C++
> software . Java is a piece of cake.
> We have built the new war from the source version 4.10.3 and our
> preliminary tests have shown that our issue (replicas in recovery on high
> load)* is resolved *. We will continue to do more testing and confirm .
> Please note that the *patch is BUGGY*.
>
> It removed the break statement within while loop because of which, whenever
> we send a list of docs it would hang (API CloudSolrServer.add) , but it
> would work if send one doc at a time.
>
> It took a while to figure out why that is happening. Once we put the break
> statement back it worked like a charm.
> Furthermore the patch has
>
> solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
> which should be
>
> solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java
>
> Finally checking if(!offer) is sufficient than using if(offer == false)
> Last but not the least having a configurable queue size and timeouts
> (managed via solrconfig) would be quite helpful
> Thank you once again for your help.
>
> Vijay
>
> On Tue, Jan 27, 2015 at 6:20 PM, Shawn Heisey <ap...@elyograg.org> wrote:
>
> > On 1/27/2015 2:52 PM, Vijay Sekhri wrote:
> > > Hi Shawn,
> > > Here is some update. We found the main issue
> > > We have configured our cluster to run under jetty and when we tried
> full
> > > indexing, we did not see the original Invalid Chunk error. However the
> > > replicas still went into recovery
> > > All this time we been trying to look into replicas logs to diagnose the
> > > issue. The problem seem to be at the leader side. When we looked into
> > > leader logs, we found the following on all the leaders
> > >
> > > 3439873 [qtp1314570047-92] WARN
> > >  org.apache.solr.update.processor.DistributedUpdateProcessor  – Error
> > > sending update
> > > *java.lang.IllegalStateException: Queue full*
> >
> > <snip>
> >
> > > There is a similar bug reported around this
> > > https://issues.apache.org/jira/browse/SOLR-5850
> > >
> > > and it seem to be in OPEN status. Is there a way we can configure the
> > queue
> > > size and increase it ? or is there a version of solr that has this
> issue
> > > resolved already?
> > > Can you suggest where we go from here to resolve this ? We can repatch
> > the
> > > war file if that is what you would recommend .
> > > In the end our initial speculation about solr unable to handle so many
> > > update is correct. We do not see this issue when the update load is
> less.
> >
> > Are you in a position where you can try the patch attached to
> > SOLR-5850?  You would need to get the source code for the version you're
> > on (or perhaps a newer 4.x version), patch it, and build Solr yourself.
> > If you have no experience building java packages from source, this might
> > prove to be difficult.
> >
> > Thanks,
> > Shawn
> >
> >
>
>
> --
> *********************************************
> Vijay Sekhri
> *********************************************
>

Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Shawn,
Thank you so much for the assistance. Building is not a problem . Back in
the days I have worked with linking, compiling and  building C , C++
software . Java is a piece of cake.
We have built the new war from the source version 4.10.3 and our
preliminary tests have shown that our issue (replicas in recovery on high
load)* is resolved *. We will continue to do more testing and confirm .
Please note that the *patch is BUGGY*.

It removed the break statement within while loop because of which, whenever
we send a list of docs it would hang (API CloudSolrServer.add) , but it
would work if send one doc at a time.

It took a while to figure out why that is happening. Once we put the break
statement back it worked like a charm.
Furthermore the patch has
solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java
which should be
solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java

Finally checking if(!offer) is sufficient than using if(offer == false)
Last but not the least having a configurable queue size and timeouts
(managed via solrconfig) would be quite helpful
Thank you once again for your help.

Vijay

On Tue, Jan 27, 2015 at 6:20 PM, Shawn Heisey <ap...@elyograg.org> wrote:

> On 1/27/2015 2:52 PM, Vijay Sekhri wrote:
> > Hi Shawn,
> > Here is some update. We found the main issue
> > We have configured our cluster to run under jetty and when we tried full
> > indexing, we did not see the original Invalid Chunk error. However the
> > replicas still went into recovery
> > All this time we been trying to look into replicas logs to diagnose the
> > issue. The problem seem to be at the leader side. When we looked into
> > leader logs, we found the following on all the leaders
> >
> > 3439873 [qtp1314570047-92] WARN
> >  org.apache.solr.update.processor.DistributedUpdateProcessor  – Error
> > sending update
> > *java.lang.IllegalStateException: Queue full*
>
> <snip>
>
> > There is a similar bug reported around this
> > https://issues.apache.org/jira/browse/SOLR-5850
> >
> > and it seem to be in OPEN status. Is there a way we can configure the
> queue
> > size and increase it ? or is there a version of solr that has this issue
> > resolved already?
> > Can you suggest where we go from here to resolve this ? We can repatch
> the
> > war file if that is what you would recommend .
> > In the end our initial speculation about solr unable to handle so many
> > update is correct. We do not see this issue when the update load is less.
>
> Are you in a position where you can try the patch attached to
> SOLR-5850?  You would need to get the source code for the version you're
> on (or perhaps a newer 4.x version), patch it, and build Solr yourself.
> If you have no experience building java packages from source, this might
> prove to be difficult.
>
> Thanks,
> Shawn
>
>


-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Shawn Heisey <ap...@elyograg.org>.
On 1/27/2015 2:52 PM, Vijay Sekhri wrote:
> Hi Shawn,
> Here is some update. We found the main issue
> We have configured our cluster to run under jetty and when we tried full
> indexing, we did not see the original Invalid Chunk error. However the
> replicas still went into recovery
> All this time we been trying to look into replicas logs to diagnose the
> issue. The problem seem to be at the leader side. When we looked into
> leader logs, we found the following on all the leaders
>
> 3439873 [qtp1314570047-92] WARN
>  org.apache.solr.update.processor.DistributedUpdateProcessor  – Error
> sending update
> *java.lang.IllegalStateException: Queue full*

<snip>

> There is a similar bug reported around this
> https://issues.apache.org/jira/browse/SOLR-5850
>
> and it seem to be in OPEN status. Is there a way we can configure the queue
> size and increase it ? or is there a version of solr that has this issue
> resolved already?
> Can you suggest where we go from here to resolve this ? We can repatch the
> war file if that is what you would recommend .
> In the end our initial speculation about solr unable to handle so many
> update is correct. We do not see this issue when the update load is less.

Are you in a position where you can try the patch attached to
SOLR-5850?  You would need to get the source code for the version you're
on (or perhaps a newer 4.x version), patch it, and build Solr yourself. 
If you have no experience building java packages from source, this might
prove to be difficult.

Thanks,
Shawn


Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Shawn,
Here is some update. We found the main issue
We have configured our cluster to run under jetty and when we tried full
indexing, we did not see the original Invalid Chunk error. However the
replicas still went into recovery
All this time we been trying to look into replicas logs to diagnose the
issue. The problem seem to be at the leader side. When we looked into
leader logs, we found the following on all the leaders

3439873 [qtp1314570047-92] WARN
 org.apache.solr.update.processor.DistributedUpdateProcessor  – Error
sending update
*java.lang.IllegalStateException: Queue full*
        at java.util.AbstractQueue.add(AbstractQueue.java:98)
        at
org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrServer$Runner$1.writeTo(ConcurrentUpdateSolrServer.java:182)
        at
org.apache.http.entity.EntityTemplate.writeTo(EntityTemplate.java:69)
        at
org.apache.http.entity.HttpEntityWrapper.writeTo(HttpEntityWrapper.java:89)
        at
org.apache.http.impl.client.EntityEnclosingRequestWrapper$EntityWrapper.writeTo(EntityEnclosingRequestWrapper.java:108)
        at
org.apache.http.impl.entity.EntitySerializer.serialize(EntitySerializer.java:117)
        at
org.apache.http.impl.AbstractHttpClientConnection.sendRequestEntity(AbstractHttpClientConnection.java:265)
        at
org.apache.http.impl.conn.ManagedClientConnectionImpl.sendRequestEntity(ManagedClientConnectionImpl.java:203)
        at
org.apache.http.protocol.HttpRequestExecutor.doSendRequest(HttpRequestExecutor.java:236)
        at
org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:121)
        at
org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:682)
        at
org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:486)
        at
org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:863)
        at
org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
        at
org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:106)
        at
org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:57)
        at
org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrServer$Runner.run(ConcurrentUpdateSolrServer.java:233)
        at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)
3439874 [qtp1314570047-92] INFO  org.apache.solr.common.cloud.SolrZkClient
 – makePath:
/collections/search1/leader_initiated_recovery/shard7/core_node214
3439879 [qtp1314570047-92] INFO  org.apache.solr.cloud.ZkController  –
Wrote down to
/collections/search1/leader_initiated_recovery/shard7/core_node214
3439879 [qtp1314570047-92] INFO  org.apache.solr.cloud.ZkController  – Put
replica core=search1_shard7_replica1 coreNodeName=core_node214 on
XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX:8580_solr into leader-initiated
recovery.
3439880 [qtp1314570047-92] WARN  org.apache.solr.cloud.ZkController  –
Leader is publishing core=search1_shard7_replica1 coreNodeName
=core_node214 state=down on behalf of un-reachable replica
http://XXXXXXXXXXXXXXX:YYYYYY/solr/search1_shard7_replica1/;
forcePublishState? false
3439881 [zkCallback-2-thread-12] INFO
 org.apache.solr.cloud.DistributedQueue  – LatchChildWatcher fired on path:
/overseer/queue state: SyncConnected type NodeChildrenChanged
3439881 [qtp1314570047-92] ERROR
org.apache.solr.update.processor.DistributedUpdateProcessor  – *Setting up
to try to start recovery on replica
http://XXXXXXXXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYY/solr/search1_shard7_replica1/
after: java.lang.IllegalStateException: Queue full*
3439882 [qtp1314570047-92] INFO  org.apache.solr.core.SolrCore  –
[search1_shard7_replica2] webapp=/solr path=/update
params={wt=javabin&version=2} status=0 QTime=2608
3439882 [updateExecutor-1-thread-153] INFO
 org.apache.solr.cloud.LeaderInitiatedRecoveryThread  –
LeaderInitiatedRecoveryThread-search1_shard7_replica1 started running to
send REQUESTRECOVERY command to
http://XXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYY/solr/search1_shard7_replica1/;
will try for a max of 600 secs
3439882 [updateExecutor-1-thread-153] INFO
 org.apache.solr.cloud.LeaderInitiatedRecoveryThread  – Asking
core=search1_shard7_replica1 coreNodeName=core_node214 on
http://XXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYY/solr to recover
3439885
[OverseerStateUpdate-93213309377511456-XXXXXXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYY_solr-n_0000002822]
INFO  org.apache.solr.cloud.Overseer


There is a similar bug reported around this
https://issues.apache.org/jira/browse/SOLR-5850

and it seem to be in OPEN status. Is there a way we can configure the queue
size and increase it ? or is there a version of solr that has this issue
resolved already?
Can you suggest where we go from here to resolve this ? We can repatch the
war file if that is what you would recommend .
In the end our initial speculation about solr unable to handle so many
update is correct. We do not see this issue when the update load is less.




Here is the log from the replica FYI. No invalid chunk error with Jetty but
the issue at hand is unrelated

2382763 [Thread-74] INFO  org.apache.solr.cloud.ZkController  – Wrote
recovering to /collections/search1/leader_initiated_recovery
/shard7/core_node214
2382776 [Thread-74] INFO  org.apache.solr.update.DefaultSolrCoreState  –
Running recovery - first canceling any ongoing recovery
2382777 [Thread-74] WARN  org.apache.solr.cloud.RecoveryStrategy  –
Stopping recovery for core=search1_shard7_replica1 coreNodeName=core_node214
2382778 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Starting recovery process.  core=search1_shard7_replica1
recoveringAfterStartup=false
2382779 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Publishing state of core search1_shard7_replica1 as recovering, leader is
http://XXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYYYYYY/solr/search1_shard7_replica2/
and I am http://XXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYYYYYY
/solr/search1_shard7_replica1/
2382779 [RecoveryThread] INFO  org.apache.solr.cloud.ZkController  –
publishing core=search1_shard7_replica1 state=recovering collection=search1
2382779 [RecoveryThread] INFO  org.apache.solr.cloud.ZkController  –
numShards not found on descriptor - reading it from system property
2382785 [zkCallback-2-thread-5] INFO
 org.apache.solr.common.cloud.ZkStateReader  – A cluster state change:
WatchedEvent state:SyncConnected type:NodeDataChanged
path:/clusterstate.json, has occurred - updating... (live nodes size: 40)
2382796 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Sending prep recovery command to http://XXXXXXXXXXXXXX:YYYYYYYYYYYYYYYY/solr;
WaitForState:
action=PREPRECOVERY&core=search1_shard7_replica2&nodeName=XXXXXXXXXXXXXXXXXXXXXX%3AXXXXXXXXXXX_solr&coreNodeName=core_node214&state=recovering&checkLive=true&onlyIfLeader=true&onlyIfLeaderActive=true
2382953 [zkCallback-2-thread-5] INFO
 org.apache.solr.common.cloud.ZkStateReader  – A cluster state change:
WatchedEvent state:SyncConnected type:NodeDataChanged
path:/clusterstate.json, has occurred - updating... (live nodes size: 40)
2385815 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Attempting to PeerSync from
http://sXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYYYYYYYYYYYY/solr/search1_shard7_replica2/
core=search1_shard7_replica1 - recoveringAfterStartup=false
2385815 [RecoveryThread] INFO  org.apache.solr.update.PeerSync  – PeerSync:
core=search1_shard7_replica1 url=http://XXXXXXXXXXXXXX:YYYYYYYY/solr START
replicas=[http://XXXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYY/solr/search1_shard7_replica2/]
nUpdates=100
2385816 [RecoveryThread] WARN  org.apache.solr.update.PeerSync  – PeerSync:
core=search1_shard7_replica1 url=http://XXXXXXXXXXXXXXXX:YYYYYYYYYYYYYY/solr
too many updates received since start - startingUpdates no longer overlaps
with our currentUpdates
2385817 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
PeerSync Recovery was not successful - trying replication.
core=search1_shard7_replica1
2385817 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Starting Replication Recovery. core=search1_shard7_replica1
2385817 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Begin buffering updates. core=search1_shard7_replica1
2385817 [RecoveryThread] INFO  org.apache.solr.update.UpdateLog  – Starting
to buffer updates. FSUpdateLog{state=ACTIVE,
tlog=tlog{file=/opt/solr/solrnodes/solrnode1/search1_shard7_replica1/data/tlog/tlog.0000000000000000407
refcount=1}}
2385817 [RecoveryThread] INFO  org.apache.solr.cloud.RecoveryStrategy  –
Attempting to replicate from
http://XXXXXXXXXXXXXXXXXXXXXX:YYYYYYYYYYYYY/solr/search1_shard7_replica2/.
core=search1_shard7_replica1
2386469 [RecoveryThread] INFO  org.apache.solr.handler.SnapPuller  –  No
value set for 'pollInterval'. Timer Task not started.
2386477 [RecoveryThread] INFO  org.apache.solr.handler.SnapPuller  –
Master's generation: 963
2386478 [RecoveryThread] INFO  org.apache.solr.handler.SnapPuller  –
Slave's generation: 962
2386478 [RecoveryThread] INFO  org.apache.solr.handler.SnapPuller  –
Starting replication process
2386489 [RecoveryThread] INFO  org.apache.solr.handler.SnapPuller  – Number
of files in latest index in master: 250
2386489 [RecoveryThread] INFO  org.apache.solr.core.CachingDirectoryFactory
 – return new directory for
/opt/solr/solrnodes/solrnode1/search1_shard7_replica1/data/index.20150127135840219
2386491 [RecoveryThread] WARN  org.apache.solr.handler.SnapPuller  – File
_89e.fdt expected to be 811432 while it is 4204287

-regards
Vijay


On Tue, Jan 27, 2015 at 12:07 AM, Shawn Heisey <ap...@elyograg.org> wrote:

> On 1/26/2015 9:34 PM, Vijay Sekhri wrote:
> > Hi Shawn, Erick
> > So it turned out that once we increased our indexing rate to the original
> > full indexing rate  the replicas went back into recovery no matter what
> the
> > zk timeout setting was. Initially we though that increasing the timeout
> is
> > helping but apparently not . We just decreased indexing rate and that
> > caused less replicas to go in recovery. Once we have our full indexing
> rate
> > almost all replicas went into recovery no matter what the zk timeout or
> the
> > ticktime setting were. We reverted back the ticktime to original 2
> seconds
> >
> > So we investigated further and after checking the logs we found this
> > exception happening right before the recovery process is initiated. We
> > observed this on two different replicas that went into recovery. We are
> not
> > sure if this is a coincidence or a real problem . Notice we were also
> > putting some search query load while indexing to trigger the recovery
> > behavior
>
> <snip>
>
> > 22:00:40,861 ERROR [org.apache.solr.core.SolrCore]
> (http-/10.235.46.36:8580-32)
> > ClientAbortException: * java.io.IOException: JBWEB002020: Invalid chunk
> > header*
>
> One possibility that my searches on that exception turned up is that
> this is some kind of a problem in the servlet container, and the
> information I can see suggests it may be a bug in JBoss, and the
> underlying cause is changes in newer releases of Java 7.  Your
> stacktraces do seem to mention jboss classes, so that seems likely.  The
> reason that we only recommend running under the Jetty that comes with
> Solr, which has a tuned config, is because that's the only servlet
> container that actually gets tested.
>
> https://bugzilla.redhat.com/show_bug.cgi?id=1104273
> https://bugzilla.redhat.com/show_bug.cgi?id=1154028
>
> I can't really verify any other possibility.
>
> Thanks,
> Shawn
>
>


-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Shawn Heisey <ap...@elyograg.org>.
On 1/26/2015 9:34 PM, Vijay Sekhri wrote:
> Hi Shawn, Erick
> So it turned out that once we increased our indexing rate to the original
> full indexing rate  the replicas went back into recovery no matter what the
> zk timeout setting was. Initially we though that increasing the timeout is
> helping but apparently not . We just decreased indexing rate and that
> caused less replicas to go in recovery. Once we have our full indexing rate
> almost all replicas went into recovery no matter what the zk timeout or the
> ticktime setting were. We reverted back the ticktime to original 2 seconds
> 
> So we investigated further and after checking the logs we found this
> exception happening right before the recovery process is initiated. We
> observed this on two different replicas that went into recovery. We are not
> sure if this is a coincidence or a real problem . Notice we were also
> putting some search query load while indexing to trigger the recovery
> behavior

<snip>

> 22:00:40,861 ERROR [org.apache.solr.core.SolrCore] (http-/10.235.46.36:8580-32)
> ClientAbortException: * java.io.IOException: JBWEB002020: Invalid chunk
> header*

One possibility that my searches on that exception turned up is that
this is some kind of a problem in the servlet container, and the
information I can see suggests it may be a bug in JBoss, and the
underlying cause is changes in newer releases of Java 7.  Your
stacktraces do seem to mention jboss classes, so that seems likely.  The
reason that we only recommend running under the Jetty that comes with
Solr, which has a tuned config, is because that's the only servlet
container that actually gets tested.

https://bugzilla.redhat.com/show_bug.cgi?id=1104273
https://bugzilla.redhat.com/show_bug.cgi?id=1154028

I can't really verify any other possibility.

Thanks,
Shawn


Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Shawn, Erick
So it turned out that once we increased our indexing rate to the original
full indexing rate  the replicas went back into recovery no matter what the
zk timeout setting was. Initially we though that increasing the timeout is
helping but apparently not . We just decreased indexing rate and that
caused less replicas to go in recovery. Once we have our full indexing rate
almost all replicas went into recovery no matter what the zk timeout or the
ticktime setting were. We reverted back the ticktime to original 2 seconds

So we investigated further and after checking the logs we found this
exception happening right before the recovery process is initiated. We
observed this on two different replicas that went into recovery. We are not
sure if this is a coincidence or a real problem . Notice we were also
putting some search query load while indexing to trigger the recovery
behavior

22:00:32,493 INFO  [org.apache.solr.cloud.RecoveryStrategy]
(rRecoveryThread) Finished recovery process. core=search1_shard5_replica2
22:00:32,503 INFO  [org.apache.solr.common.cloud.ZkStateReader]
(zkCallback-2-thread-66) A cluster state change: WatchedEvent
state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
occurred - updating... (live nodes size: 22)
22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: trigger
flush: activeBytes=101796784 deleteBytes=3061644 vs limit=104857600
22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: thread state
has 12530488 bytes; docInRAM=2051
22:00:40,450 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [FP][http-/10.235.46.36:8580-27]: thread state
has 12984633 bytes; docInRAM=2205


22:00:40,861 ERROR [org.apache.solr.core.SolrCore] (http-/10.235.46.36:8580-32)
ClientAbortException: * java.io.IOException: JBWEB002020: Invalid chunk
header*
        at
org.apache.catalina.connector.InputBuffer.realReadBytes(InputBuffer.java:351)
        at
org.apache.tomcat.util.buf.ByteChunk.substract(ByteChunk.java:422)
        at
org.apache.catalina.connector.InputBuffer.read(InputBuffer.java:373)
        at
org.apache.catalina.connector.CoyoteInputStream.read(CoyoteInputStream.java:193)
        at
org.apache.solr.common.util.FastInputStream.readWrappedStream(FastInputStream.java:80)
        at
org.apache.solr.common.util.FastInputStream.refill(FastInputStream.java:89)
        at
org.apache.solr.common.util.FastInputStream.readByte(FastInputStream.java:192)
        at
org.apache.solr.common.util.JavaBinCodec.unmarshal(JavaBinCodec.java:111)
        at
org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec.unmarshal(JavaBinUpdateRequestCodec.java:173)
        at
org.apache.solr.handler.loader.JavabinLoader.parseAndLoadDocs(JavabinLoader.java:106)
        at
org.apache.solr.handler.loader.JavabinLoader.load(JavabinLoader.java:58)
        at
org.apache.solr.handler.UpdateRequestHandler$1.load(UpdateRequestHandler.java:99)
        at
org.apache.solr.handler.ContentStreamHandlerBase.handleRequestBody(ContentStreamHandlerBase.java:74)
        at
org.apache.solr.handler.RequestHandlerBase.handleRequest(RequestHandlerBase.java:135)
        at org.apache.solr.core.SolrCore.execute(SolrCore.java:1967)
        at
org.apache.solr.servlet.SolrDispatchFilter.execute(SolrDispatchFilter.java:777)
        at
org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:418)
        at
org.apache.solr.servlet.SolrDispatchFilter.doFilter(SolrDispatchFilter.java:207)
        at
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:246)
        at
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:214)
        at
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:230)
        at
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:149)
        at
org.jboss.as.web.security.SecurityContextAssociationValve.invoke(SecurityContextAssociationValve.java:169)
        at
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:145)
        at
org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:97)
        at
org.apache.catalina.valves.AccessLogValve.invoke(AccessLogValve.java:559)
        at
org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:102)
        at
org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:336)
        at
org.apache.coyote.http11.Http11Processor.process(Http11Processor.java:856)
                                          at
org.apache.tomcat.util.net.JIoEndpoint$Worker.run(JIoEndpoint.java:920)
        at java.lang.Thread.run(Thread.java:744)
Caused by: java.io.IOException: JBWEB002020: Invalid chunk header
        at
org.apache.coyote.http11.filters.ChunkedInputFilter.parseChunkHeader(ChunkedInputFilter.java:281)
        at
org.apache.coyote.http11.filters.ChunkedInputFilter.doRead(ChunkedInputFilter.java:134)
        at
org.apache.coyote.http11.InternalInputBuffer.doRead(InternalInputBuffer.java:697)
        at org.apache.coyote.Request.doRead(Request.java:438)
        at
org.apache.catalina.connector.InputBuffer.realReadBytes(InputBuffer.java:341)
        ... 31 more
22:00:40,864 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: new
segment has 0 deleted docs
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: new
segment has no vectors; no norms; no docValues; prox; freqs
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]:
flushedFiles=[_227_Lucene41_0.tip, _227.fdx, _227_Lucene41_0.tim, _227.fdt,
_227_Lucene41_0.doc, _227.fnm, _227_Lucene41_0.pos]
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: flushed
codec=Lucene410
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DWPT][http-/10.235.46.36:8580-27]: flushed:
segment=_227 ramUsed=12.383 MB newFlushedSize(includes docstores)=4.444 MB
docs/MB=496.165
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [DW][http-/10.235.46.36:8580-27]:
publishFlushedSegment seg-private updates=null
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [IW][http-/10.235.46.36:8580-27]:
publishFlushedSegment
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [BD][http-/10.235.46.36:8580-27]: push deletes
 16553 deleted terms (unique count=16553) bytesUsed=222240 delGen=17
packetCount=1 totBytesUsed=222240
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [IW][http-/10.235.46.36:8580-27]: publish sets
newSegment delGen=18 seg=_227(4.10.0):C2205
22:00:40,865 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [IFD][http-/10.235.46.36:8580-27]: now
checkpoint "_12l(4.10.0):C676510/354682:delGen=208
_1wo(4.10.0):C420813/69819:delGen=40 _1bs(4.10.0):C229862/110117:delGen=77
_1ga(4.10.0):C322386/84434:delGen=91 _1nc(4.10.0):C209477/2949:delGen=52
_223(4.10.0):C206753/11:delGen=1 _1ug(4.10.0):C7185/1:delGen=1
_20n(4.10.0):C59384/2287:delGen=7 _1zm(4.10.0):C78927/1581:delGen=12
_20v(4.10.0):C4317 _20u(4.10.0):C5243/1:delGen=1 _20z(4.10.0):C3288
_20y(4.10.0):C4614/1:delGen=1 _21i(4.10.0):C2190 _21f(4.10.0):C16
_21g(4.10.0):C67 _21h(4.10.0):C18 _21o(4.10.0):C116
_21q(4.10.0):C1243/1:delGen=1 _21r(4.10.0):C39 _21s(4.10.0):C92
_21t(4.10.0):C1 _21u(4.10.0):C1 _222(4.10.0):C26988
_224(4.10.0):C28193/1:delGen=1 _225(4.10.0):C25966 _227(4.10.0):C2205" [27
segments ; isCommit = false]
22:00:40,866 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [IFD][http-/10.235.46.36:8580-27]: 0 msec to
checkpoint
22:00:40,866 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [TMP][http-/10.235.46.36:8580-27]: findMerges:
27 segments
10.235.46.36:8580-27]: now merge
22:00:40,867 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [CMS][http-/10.235.46.36:8580-27]:   index:
_12l(4.10.0):C676510/354682:delGen=208 _1wo(4.10.0):C420813/69819:delGen=40
_1bs(4.10.0):C229862/110117:delGen=77 _1ga(4.10.0):C322386/84434:delGen=91
_1nc(4.10.0):C209477/2949:delGen=52 _223(4.10.0):C206753/11:delGen=1
_1ug(4.10.0):C7185/1:delGen=1 _20n(4.10.0):C59384/2287:delGen=7
_1zm(4.10.0):C78927/1581:delGen=12 _20v(4.10.0):C4317
_20u(4.10.0):C5243/1:delGen=1 _20z(4.10.0):C3288
_20y(4.10.0):C4614/1:delGen=1 _21i(4.10.0):C2190 _21f(4.10.0):C16
_21g(4.10.0):C67 _21h(4.10.0):C18 _21o(4.10.0):C116
_21q(4.10.0):C1243/1:delGen=1 _21r(4.10.0):C39 _21s(4.10.0):C92
_21t(4.10.0):C1 _21u(4.10.0):C1 _222(4.10.0):C26988
_224(4.10.0):C28193/1:delGen=1 _225(4.10.0):C25966 _227(4.10.0):C2205
22:00:40,867 INFO  [org.apache.solr.update.LoggingInfoStream]
(http-/10.235.46.36:8580-27) [CMS][http-/10.235.46.36:8580-27]:   no more
merges pending; now return
22:00:41,885 INFO  [org.apache.solr.handler.admin.CoreAdminHandler]
(http-/10.235.46.36:8580-32) It has been requested that we recover:
core=search1_shard5_replica2
22:00:41,886 INFO  [org.apache.solr.servlet.SolrDispatchFilter]
(http-/10.235.46.36:8580-32) [admin] webapp=null path=/admin/cores
params={action=REQUESTRECOVERY&core=search1_shard5_replica2&wt=javabin&version=2}
status=0 QTime=1
22:00:41,886 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)
publishing core=search1_shard5_replica2 state=recovering collection=search1
22:00:41,886 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)
numShards not found on descriptor - reading it from system property

22:00:41,889 INFO  [org.apache.solr.common.cloud.ZkStateReader]
(zkCallback-2-thread-66) A cluster state change: WatchedEvent
state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
occurred - updating... (live nodes size: 22)
22:00:41,890 INFO  [org.apache.solr.cloud.ZkController] (Thread-472)* Wrote
recovering to
/collections/search1/leader_initiated_recovery/shard5/core_node223*
22:00:41,892 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(Thread-472) Running recovery - first canceling any ongoing recovery
22:00:41,893 INFO  [org.apache.solr.cloud.RecoveryStrategy]
(RecoveryThread) Starting recovery process.  core=search1_shard5_replica2
recoveringAfterStartup=false
22:00:41,894 INFO  [org.apache.solr.cloud.RecoveryStrategy]
(RecoveryThread) Publishing state of core search1_shard5_replica2 as
recovering, leader is http://XXXXXXXXXXCHANGEDONPURPOSEXXXX:8680/solr/search1/
and I am http://solrx341p.qa.ch3.s.com:8580/solr/search1_shard5_replica2/
22:00:41,894 INFO  [org.apache.solr.cloud.ZkController] (RecoveryThread)
publishing core=search1_shard5_replica2 state=recovering collection=search1
22:00:41,894 INFO  [org.apache.solr.cloud.ZkController] (RecoveryThread)
numShards not found on descriptor - reading it from system property
22:00:41,896 INFO  [org.apache.solr.cloud.RecoveryStrategy]
(RecoveryThread) Sending prep recovery command to
http://XXXXXXXXXCHANGEDONPURPOSEXXXXXXXX:8680/solr; WaitForState:
action=PREPRECOVERY&core=search1&nodeName=XXXXXXXXXXXXXXCHANGEDONPURPOSE
XXXXXXXXXXXXXXX%3A8580_solr&coreNodeName=core_node223&state=recovering&checkLive=true&onlyIfLeader=true&onlyIfLeaderActive=true
22:00:42,006 INFO  [org.apache.solr.common.cloud.ZkStateReader]
(zkCallback-2-thread-66) A cluster state change: WatchedEvent
state:SyncConnected type:NodeDataChanged path:/clusterstate.json, has
occurred - updating... (live nodes size: 22)




15Gb of heap is allocated to these searchers and it hardly goes over 8GB.
Full GC does not happen. We write gc logs to separate file and also
monitored these process via visualvm . It seems to have enough memory


On Mon, Jan 26, 2015 at 5:13 PM, Shawn Heisey <ap...@elyograg.org> wrote:

> On 1/26/2015 2:26 PM, Vijay Sekhri wrote:
> > Hi Erick,
> > In solr.xml file I had zk timeout set to/ <int
> > name="zkClientTimeout">${zkClientTimeout:450000}</int>/
> > One thing that made a it a bit better now is the zk tick time and
> > syncLimit settings. I set it to a higher value as below. This may not
> > be advisable though.
> >
> > tickTime=30000
> > initLimit=30
> > syncLimit=20
> >
> > Now we observed that replicas do not go in recovery that often as
> > before. In the whole cluster at a given time I would have a couple of
> > replicas in recovery whereas earlier it were multiple replicas from
> > every shard .
> > On the wiki https://wiki.apache.org/solr/SolrCloudit says the "The
> > maximum is 20 times the tickTime." in the FAQ so I decided to increase
> > the tick time. Is this the correct approach ?
>
> The default zkClientTimeout on recent Solr versions is 30 seconds, up
> from 15 in slightly older releases.
>
> Those values of 15 or 30 seconds are a REALLY long time in computer
> terms, and if you are exceeding that timeout on a regular basis,
> something is VERY wrong with your Solr install.  Rather than take steps
> to increase your timeout beyond the normal maximum of 40 seconds (20
> times a tickTime of 2 seconds), figure out why you're exceeding that
> timeout and fix the performance problem.  The zkClientTimeout value that
> you have set, 450 seconds, is seven and a half *MINUTES*.  Nothing in
> Solr should ever take that long.
>
> "Not enough memory in the server" is by far the most common culprit for
> performance issues.  Garbage collection pauses are a close second.
>
> I don't actually know this next part for sure, because I've never looked
> into the code, but I believe that increasing the tickTime, especially to
> a value 15 times higher than default, might make all zookeeper
> operations a lot slower.
>
> Thanks,
> Shawn
>
>


-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Shawn Heisey <ap...@elyograg.org>.
On 1/26/2015 2:26 PM, Vijay Sekhri wrote:
> Hi Erick,
> In solr.xml file I had zk timeout set to/ <int
> name="zkClientTimeout">${zkClientTimeout:450000}</int>/
> One thing that made a it a bit better now is the zk tick time and
> syncLimit settings. I set it to a higher value as below. This may not
> be advisable though. 
>
> tickTime=30000
> initLimit=30
> syncLimit=20
>
> Now we observed that replicas do not go in recovery that often as
> before. In the whole cluster at a given time I would have a couple of
> replicas in recovery whereas earlier it were multiple replicas from
> every shard .
> On the wiki https://wiki.apache.org/solr/SolrCloudit says the "The
> maximum is 20 times the tickTime." in the FAQ so I decided to increase
> the tick time. Is this the correct approach ?

The default zkClientTimeout on recent Solr versions is 30 seconds, up
from 15 in slightly older releases.

Those values of 15 or 30 seconds are a REALLY long time in computer
terms, and if you are exceeding that timeout on a regular basis,
something is VERY wrong with your Solr install.  Rather than take steps
to increase your timeout beyond the normal maximum of 40 seconds (20
times a tickTime of 2 seconds), figure out why you're exceeding that
timeout and fix the performance problem.  The zkClientTimeout value that
you have set, 450 seconds, is seven and a half *MINUTES*.  Nothing in
Solr should ever take that long.

"Not enough memory in the server" is by far the most common culprit for
performance issues.  Garbage collection pauses are a close second.

I don't actually know this next part for sure, because I've never looked
into the code, but I believe that increasing the tickTime, especially to
a value 15 times higher than default, might make all zookeeper
operations a lot slower.

Thanks,
Shawn


Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Erick,
In solr.xml file I had zk timeout set to*  <int
name="zkClientTimeout">${zkClientTimeout:450000}</int>*
One thing that made a it a bit better now is the zk tick time and syncLimit
settings. I set it to a higher value as below. This may not be advisable
though.

tickTime=30000
initLimit=30
syncLimit=20

Now we observed that replicas do not go in recovery that often as before.
In the whole cluster at a given time I would have a couple of replicas in
recovery whereas earlier it were multiple replicas from every shard .
On the wiki https://wiki.apache.org/solr/SolrCloud it says the "The maximum
is 20 times the tickTime." in the FAQ so I decided to increase the tick
time. Is this the correct approach ?

One question I have is that if auto commit settings has anything to do with
this or not ? Does it induce extra work for the searchers because of which
this would happen? I have tried with following settings
*  <autoSoftCommit>*
*            <maxDocs>500000</maxDocs>*
*            <maxTime>900000</maxTime>*
*        </autoSoftCommit>*

*        <autoCommit>*
*            <maxDocs>200000</maxDocs>*
*            <maxTime>30000</maxTime>*
*            <openSearcher>false</openSearcher>*
*        </autoCommit>*

I have increased  the  heap size to 15GB for each JVM instance . I
monitored during full indexing how the heap usage looks like and it never
goes beyond 8 GB . Herewith attached  are a few screen shot of heap usage
accumulated during indexing and I don't see any Full GC happening at any
point



Our rate is a variable rate . It is not a sustained rate of 6000/second ,
however there are intervals where it would reach that much and come down
and grow again and come down.  So if I would take an average it would be
600/second only but that is not real rate at any given time.
Version of solr cloud is 4.10.  All indexers are basically java programs
running on different host using CloudSolrServer api.
As I mentioned it is much better now than before , however not completely
as expected . We would want none of them to go in recovery if really there
is no need.

I captured some logs before and after recovery

 4:13:54,298 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
New index installed. Updating index properties...
index=index.20150126140904697
14:13:54,301 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
removing old index directory
NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417;
maxCacheMB=48.0 maxMergeSizeMB=4.0)
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Creating new IndexWriter...
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Waiting until IndexWriter is unused...
core=search1_shard7_replica4
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Rollback old IndexWriter... core=search1_shard7_replica4
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: all running merges have aborted
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback: done finish merges
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: abort
14:13:54,303 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: done abort; abortedFiles=[]
success=true
14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback:
infos=_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
_5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
_5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
_5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
_5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
_5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
_5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122
14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: now checkpoint
"_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
_5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
_5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
_5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
_5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
_5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
_5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122" [30 segments ;
isCommit = false]
14:13:54,307 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: 0 msec to checkpoint
14:13:54,323 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread) New
index directory detected:
old=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
new=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
14:13:54,417 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: init: current segments file is
"segments_9t";
deletionPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper@592ddff6
14:13:54,420 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: init: load commit "segments_9t"
14:13:54,424 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread)
SolrDeletionPolicy.onInit: commits: num=1
        commit{dir=NRTCachingDirectory(MMapDirectory@
/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
ersion=4.10.0
matchVersion=4.8.0
analyzer=null
ramBufferSizeMB=100.0
maxBufferedDocs=-1
maxBufferedDeleteTerms=-1
mergedSegmentWarmer=null
readerTermsIndexDivisor=1
termIndexInterval=32
delPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper
commit=null
openMode=APPEND
similarity=com.shc.solrx.similarity.CustomSimilarity
mergeScheduler=ConcurrentMergeScheduler: maxThreadCount=1, maxMergeCount=2,
mergeThreadPriority=-1
default WRITE_LOCK_TIMEOUT=1000
writeLockTimeout=1000
codec=Lucene410
infoStream=org.apache.solr.update.LoggingInfoStream
mergePolicy=[TieredMergePolicy: maxMergeAtOnce=10,
maxMergeAtOnceExplicit=30, maxMergedSegmentMB=5120.0, floorSegmentMB=2.0,
forceMergeDeletesPctAllowed=10.0, segmentsPerTier=10.0,
maxCFSSegmentSizeMB=8.796093022207999E12, noCFSRatio=0.0
indexerThreadPool=org.apache.lucene.index.DocumentsWriterPerThreadPool@693825c0
readerPooling=false
perThreadHardLimitMB=1945
useCompoundFile=false
checkIntegrityAtMerge=false
writer=org.apache.lucene.util.SetOnce@65369637

14:13:54,425 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) New IndexWriter is ready to be used.
14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: flush at getReader
14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: startFullFlush
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: anyChanges? numDocsInRam=0
deletes=false hasTickets:false pendingChangesInFullFlush: false
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: apply all deletes during flush
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [BD][RecoveryThread]: applyDeletes: no deletes; skipping
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [BD][RecoveryThread]: prune sis=segments_9t:
_4qe(4.10.0):C4312879/1370002

lrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates}
status=0 QTime=0
14:16:49,279 INFO  [org.apache.solr.core.SolrCore] (http-/10.235.47.41:8580-1)
[search1_shard7_replica4] webapp=/solr path=/update params={distrib.from=
http://solrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates}
status=0 QTime=0
14:16:49,283 INFO  [org.apache.solr.update.UpdateHandler]
(recoveryExecutor-7-thread-1) start
commit{flags=2,optimize=false,openSearcher=true,waitSearcher=true,expungeDeletes=false,softCommit=false,prepareCommit=false}
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
start
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
enter lock
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
now prepare
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
prepareCommit: flush
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:   index
before flush _4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
_5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
_5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
_5r4(4.10.0):C23934/1:delGen=1
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
startFullFlush
14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=24222 deletes=true hasTickets:false
pendingChangesInFullFlush: false
14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWFC][recoveryExecutor-7-thread-1]:
addFlushableState DocumentsWriterPerThread [pendingDeletes=gen=0 24221
deleted terms (unique count=24220) bytesUsed=4455794, segment=_5r5,
aborting=false, numDocsInRAM=24222, deleteQueue=DWDQ: [ generation: 1 ]]
14:16:49,322 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush
postings as segment _5r5 numDocs=24222

==> gc.20150126-135638.log <==
1211.362: [GC1211.362: [ParNew: 966947K->88429K(996800K), 0.0191260 secs]
1499845K->633603K(1995752K), 0.0192710 secs] [Times: user=0.20 sys=0.00,
real=0.02 secs]

==> server.log <==
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
segment has 1 deleted docs
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
segment has no vectors; no norms; no docValues; prox; freqs
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]:
flushedFiles=[_5r5_Lucene41_0.pos, _5r5.fdx, _5r5.fnm, _5r5.fdt,
_5r5_Lucene41_0.tim, _5r5_Lucene41_0.tip, _5r5_Lucene41_0.doc]
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed
codec=Lucene410
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed:
segment=_5r5 ramUsed=75.564 MB newFlushedSize(includes docstores)=19.546 MB
docs/MB=1,239.201
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush:
write 1 deletes gen=-1
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
publishFlushedSegment seg-private updates=null
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
publishFlushedSegment
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: push
deletes  24222 deleted terms (unique count=24221) bytesUsed=286752 delGen=8
packetCount=4 totBytesUsed=1259648
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: publish
sets newSegment delGen=9 seg=_5r5(4.10.0):C24222/1:delGen=1
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
_5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
_5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
_5r4(4.10.0):C23934/1:delGen=1 _5r5(4.10.0):C24222/1:delGen=1" [34 segments
; isCommit = false]
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
deletes during flush
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
applyDeletes: infos=[_4qe(4.10.0):C4312879/1370002:delGen=56,
_554(4.10.0):C3995865/780418:delGen=23,
_56u(4.10.0):C286775/11906:delGen=15, _5co(4.10.0):C871785/93841:delGen=10,
_5m7(4.10.0):C122852/31675:delGen=11, _5hm(4.10.0):C457977/32535:delGen=11,
_5q0(4.10.0):C13610/649:delGen=6, _5kb(4.10.0):C424868/19149:delGen=11,
_5f5(4.10.0):C116528/42495:delGen=1, _5nx(4.10.0):C33236/20668:delGen=1,
_5qm(4.10.0):C29770/1:delGen=1, _5o8(4.10.0):C27155/7531:delGen=1,
_5of(4.10.0):C38545/5677:delGen=1, _5p7(4.10.0):C37457/648:delGen=1,
_5qv(4.10.0):C3973, _5q1(4.10.0):C402/1:delGen=1, _5q2(4.10.0):C779,
_5qa(4.10.0):C967, _5qc(4.10.0):C1828, _5qh(4.10.0):C1765,
_5qi(4.10.0):C1241, _5qq(4.10.0):C1997, _5qr(4.10.0):C1468,
_5qp(4.10.0):C1729, _5qo(4.10.0):C3456/1:delGen=1, _5qu(4.10.0):C27,
_5qt(4.10.0):C30, _5qx(4.10.0):C638, _5qy(4.10.0):C1407, _5qw(4.10.0):C802,
_5r2(4.10.0):C32769/1:delGen=1, _5r3(4.10.0):C26057,
_5r4(4.10.0):C23934/1:delGen=1, _5r5(4.10.0):C24222/1:delGen=1]
packetCount=4
14:16:50,402 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5r4(4.10.0):C23934/1:delGen=1 segGen=7 coalesced
deletes=[CoalescedUpdates(termSets=1,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=2
14:16:50,442 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5r3(4.10.0):C26057 segGen=5 coalesced
deletes=[CoalescedUpdates(termSets=2,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:50,487 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5r2(4.10.0):C32769/1:delGen=1 segGen=3 coalesced
deletes=[CoalescedUpdates(termSets=3,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=3
14:16:50,556 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qw(4.10.0):C802 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:50,628 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qy(4.10.0):C1407 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:50,698 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qx(4.10.0):C638 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:50,764 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qt(4.10.0):C30 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:50,824 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qu(4.10.0):C27 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:50,893 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qo(4.10.0):C3456/1:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:50,963 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qp(4.10.0):C1729 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:51,033 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qr(4.10.0):C1468 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,108 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qq(4.10.0):C1997 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,184 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qi(4.10.0):C1241 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,258 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qh(4.10.0):C1765 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:51,336 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qc(4.10.0):C1828 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:51,415 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qa(4.10.0):C967 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,482 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5q2(4.10.0):C779 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,545 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5q1(4.10.0):C402/1:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,621 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qv(4.10.0):C3973 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:51,688 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5p7(4.10.0):C37457/648:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:51,755 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5of(4.10.0):C38545/5677:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,903 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5o8(4.10.0):C27155/7531:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:51,991 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5qm(4.10.0):C29770/1:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=3
14:16:52,062 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5nx(4.10.0):C33236/20668:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1
14:16:52,126 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5f5(4.10.0):C116528/42495:delGen=1 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1126
14:16:52,205 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5kb(4.10.0):C424868/19149:delGen=11 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=30423
14:16:52,278 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5q0(4.10.0):C13610/649:delGen=6 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=1976
14:16:52,362 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5hm(4.10.0):C457977/32535:delGen=11 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=50818
14:16:52,439 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5m7(4.10.0):C122852/31675:delGen=11 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=22335
14:16:52,516 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5co(4.10.0):C871785/93841:delGen=10 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=154
14:16:52,600 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_56u(4.10.0):C286775/11906:delGen=15 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:52,659 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_554(4.10.0):C3995865/780418:delGen=23 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=0
14:16:52,766 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_4qe(4.10.0):C4312879/1370002:delGen=56 segGen=0 coalesced
deletes=[CoalescedUpdates(termSets=4,queries=0,numericDVUpdates=0,binaryDVUpdates=0)]
newDelCount=111
14:16:52,766 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
applyDeletes took 2378 msec

....
....
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"_5r2_1.del"
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1" [34 segments ; isCommit = false]
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"_5r4_1.del"
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
recoveryExecutor-7-thread-1 finishFullFlush success=true
14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
findMerges: 34 segments
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
large]
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
large]
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r3(4.10.0):C26057 size=23.893 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qq(4.10.0):C1997 size=1.781 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q2(4.10.0):C779 size=1.554 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qr(4.10.0):C1468 size=1.390 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qp(4.10.0):C1729/1:delGen=1 size=1.351 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qa(4.10.0):C967 size=1.235 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qi(4.10.0):C1241 size=1.146 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qy(4.10.0):C1407 size=1.050 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q1(4.10.0):C402/1:delGen=1 size=0.954 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qw(4.10.0):C802/1:delGen=1 size=0.821 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qx(4.10.0):C638 size=0.818 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qt(4.10.0):C30 size=0.072 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qu(4.10.0):C27 size=0.063 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
allowedSegmentCount=31 vs count=34 (eligible count=32) tooBigCount=2
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5co(4.10.0):C871785/93995:delGen=11
_5kb(4.10.0):C424868/49572:delGen=12 _5hm(4.10.0):C457977/83353:delGen=12
_56u(4.10.0):C286775/11906:delGen=15 _5f5(4.10.0):C116528/43621:delGen=2
_5m7(4.10.0):C122852/54010:delGen=12 _5p7(4.10.0):C37457/649:delGen=2
_5of(4.10.0):C38545/5677:delGen=1 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 score=0.7313777596572674 skew=0.341
nonDelRatio=0.852 tooLarge=false size=2506.203 MB
.....

14:16:52,781 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5q0(4.10.0):C13610/2625:delGen=7 _5qv(4.10.0):C3973/1:delGen=1
_5qo(4.10.0):C3456/2:delGen=2 _5qq(4.10.0):C1997 _5q2(4.10.0):C779
_5qh(4.10.0):C1765/1:delGen=1 _5qc(4.10.0):C1828/1:delGen=1
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
score=0.73678469171146 skew=0.378 nonDelRatio=0.907 tooLarge=false
size=29.296 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qv(4.10.0):C3973/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qq(4.10.0):C1997 _5q2(4.10.0):C779 _5qh(4.10.0):C1765/1:delGen=1
_5qc(4.10.0):C1828/1:delGen=1 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
score=0.3484621269100085 skew=0.151 nonDelRatio=1.000 tooLarge=false
size=17.961 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qo(4.10.0):C3456/2:delGen=2 _5qq(4.10.0):C1997 _5q2(4.10.0):C779
_5qh(4.10.0):C1765/1:delGen=1 _5qc(4.10.0):C1828/1:delGen=1
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 score=0.34135027553825914 skew=0.149
nonDelRatio=1.000 tooLarge=false size=15.609 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qq(4.10.0):C1997 _5q2(4.10.0):C779 _5qh(4.10.0):C1765/1:delGen=1
_5qc(4.10.0):C1828/1:delGen=1 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1 score=0.22756236065520313
skew=0.100 nonDelRatio=1.000 tooLarge=false size=13.416 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5q2(4.10.0):C779 _5qh(4.10.0):C1765/1:delGen=1
_5qc(4.10.0):C1828/1:delGen=1 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 score=0.22666923646436357 skew=0.100
nonDelRatio=1.000 tooLarge=false size=12.456 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qh(4.10.0):C1765/1:delGen=1 _5qc(4.10.0):C1828/1:delGen=1
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 score=0.22596772893110906
skew=0.100 nonDelRatio=1.000 tooLarge=false size=11.721 MB
14:16:52,782 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qc(4.10.0):C1828/1:delGen=1 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
score=0.224458259771065 skew=0.100 nonDelRatio=1.000 tooLarge=false
size=10.245 MB
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 score=0.22289855634027114 skew=0.100 nonDelRatio=1.000
tooLarge=false size=8.907 MB
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:   add
merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 size=8.907 MB score=0.223 skew=0.100 nonDelRatio=1.000
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
allowedSegmentCount=31 vs count=34 (eligible count=22) tooBigCount=2
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: add merge
to pendingMerges: _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
_5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
_5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
_5qt(4.10.0):C30 _5qu(4.10.0):C27 [total 1 pending]
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge merging= []
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qr(4.10.0):C1468
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qp(4.10.0):C1729/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qa(4.10.0):C967
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qi(4.10.0):C1241
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qy(4.10.0):C1407
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5q1(4.10.0):C402/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qw(4.10.0):C802/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qx(4.10.0):C638
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qt(4.10.0):C30
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qu(4.10.0):C27
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: now merge
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   index:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
consider merge _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
_5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
_5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
_5qt(4.10.0):C30 _5qu(4.10.0):C27
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
launch new thread [Lucene Merge Thread #0]
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: set
priority of merge thread Lucene Merge Thread #0 to 6
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   no more
merges pending; now return
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
startCommit(): start
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
startCommit index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 changeCount=28
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: start
14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [BD][Lucene Merge Thread #0]: applyDeletes: no deletes; skipping
14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [BD][Lucene Merge Thread #0]: prune sis=segments_9t:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
14:16:52,788 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge seg=_5r6 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:52,789 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: now merge
  merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
  index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merging _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qr(4.10.0):C1468 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qp(4.10.0):C1729/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qa(4.10.0):C967 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qi(4.10.0):C1241 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qy(4.10.0):C1407 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5q1(4.10.0):C402/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qw(4.10.0):C802/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qx(4.10.0):C638 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qt(4.10.0):C30 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qu(4.10.0):C27 no deletes
14:16:52,792 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: merge store matchedCount=10 vs 10
14:16:52,929 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 137 msec to merge stored fields
[8708 docs]
14:16:53,012 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: done all
syncs: [_5p7_Lucene41_0.pos, _5qr.si, _5kb_Lucene41_0.tim, _5q1.fdt,
_5kb_Lucene41_0.tip, _5qp.fnm, _5o8_1.del, _5qa_Lucene41_0.doc, _5qq.fnm,
_5qr.fnm, _5co_b.del, _5qh.fdx, _5qm.fdx, _5qh.fdt, _5kb_c.del, _5r5.fnm,
_554.fdx, _5qm.fdt, _5r3_Lucene41_0.doc, _4qe.fdt, _56u_Lucene41_0.doc,
_554.fdt, _5qi.fdt, _5q2_Lucene41_0.doc, _4qe.fdx, _5qa.si, _5qi.fdx, _
5qy.si, _56u.fdx, _5qv_Lucene41_0.tip, _5qh.si, _5qw_Lucene41_0.pos,
_5hm_Lucene41_0.pos, _5nx.si, _5co_Lucene41_0.pos, _5q1.fdx, _5qt.fdx,
_5qt.fdt, _5r5.si, _5co.fdt, _5nx.fnm, _5qv_Lucene41_0.tim, _5co.fdx,
_5qh_Lucene41_0.doc, _5kb.fdx, _5qh_1.del, _5qm_Lucene41_0.tim, _56u.fdt,
_5r4_2.del, _5r3_Lucene41_0.tip, _5hm.fdt, _5qv_1.del, _5qm_Lucene41_0.tip,
_5qr_Lucene41_0.doc, _5kb.fdt, _5of_Lucene41_0.tip, _5hm.fdx,
_5r3_Lucene41_0.tim, _5of_Lucene41_0.tim, _56u_f.del, _5qy_Lucene41_0.doc,
_5r4_Lucene41_0.pos, _5qq.si, _5qt_Lucene41_0.doc, _5qq_Lucene41_0.pos,
_5qa.fnm, _5qv.si, _5r2.fnm, _5qo_2.del, _5p7.fdt, _5r3.fnm,
_5p7_Lucene41_0.doc, _5r4.fdt, _5qo.si, _5p7.fdx, _5qa_Lucene41_0.tim, _
5qp.si, _5kb.si, _5r4.fdx, _5qp_Lucene41_0.tip, _4qe.si,
_5qp_Lucene41_0.tim, _5qa_Lucene41_0.tip, _4qe_1l.del, _5qp_1.del,
_5qw_1.del, _5qm_2.del, _5m7_Lucene41_0.tim, _5qw.si, _5m7_Lucene41_0.tip, _
56u.si, _5of_Lucene41_0.doc, _5of.si, _5of.fnm, _4qe_Lucene41_0.tip,
_5qh_Lucene41_0.pos, _5qi.si, _4qe_Lucene41_0.tim, _5qc.fdx,
_5r2_Lucene41_0.pos, _5qm.fnm, _5qc.fdt, _5qx_Lucene41_0.pos,
_5qo_Lucene41_0.doc, _5q0.fnm, _5qv.fdx, _5qi_Lucene41_0.pos,
_5p7_Lucene41_0.tim, _5qv.fdt, _5f5.si, _5p7_Lucene41_0.tip, _5q2.fdt, _
5o8.si, _5qm_Lucene41_0.doc, _5qc_Lucene41_0.tim, _5f5_Lucene41_0.tim,
_5f5_Lucene41_0.tip, _5qc_Lucene41_0.tip, _5qx.fdx, _554_Lucene41_0.doc,
_5q2_Lucene41_0.pos, _5r4_Lucene41_0.tim, _5qx.fdt, _5q1_Lucene41_0.pos,
_5qp_Lucene41_0.doc, _5qm.si, _56u.fnm, _5q0_Lucene41_0.pos,
_5r4_Lucene41_0.tip, _5qq.fdx, _5qr_Lucene41_0.pos, _56u_Lucene41_0.pos,
_5r2_Lucene41_0.doc, _5qw_Lucene41_0.doc, _5r5_Lucene41_0.tim,
_5r5_Lucene41_0.tip, _554.fnm, _5qi.fnm, _56u_Lucene41_0.tim,
_5qo_Lucene41_0.pos, _56u_Lucene41_0.tip, _5f5.fnm, _5qq.fdt, _5q0_7.del,
_5m7.fnm, _5qo.fnm, _5qu.si, _5q2.fdx, _5qt_Lucene41_0.tip,
_5qw_Lucene41_0.tip, _5qt_Lucene41_0.tim, _5qc.si, _5hm.si, _5m7.si, _5r2.si,
_5f5_Lucene41_0.pos, _5qy_Lucene41_0.tip, _5nx_Lucene41_0.tim,
_5r2_Lucene41_0.tip, _5qm_Lucene41_0.pos, _5qq_Lucene41_0.tim, _5co.si,
_5r2.fdt, _5r2_Lucene41_0.tim, _5f5_2.del, _5qq_Lucene41_0.tip,
_5qy_Lucene41_0.tim, _5nx_Lucene41_0.tip, _5o8_Lucene41_0.doc, _5r3.fdt,
_5qv_Lucene41_0.pos, _5q1_Lucene41_0.doc, _5qo_Lucene41_0.tip, _5qx.fnm,
_5qo_Lucene41_0.tim, _5co.fnm, _5qu.fdt, _5qu.fdx, _5r5.fdx, _554_n.del,
_4qe_Lucene41_0.pos, _5q1_Lucene41_0.tip, _5f5_Lucene41_0.doc, _5r5.fdt,
_5qw_Lucene41_0.tim, _5m7_c.del, _5qv.fnm, _5q1_Lucene41_0.tim, _5kb.fnm, _
5q2.si, _5qy.fnm, _5kb_Lucene41_0.pos, _5qx_Lucene41_0.doc, _5r4.fnm,
_5qu_Lucene41_0.pos, _5r5_Lucene41_0.doc, _554_Lucene41_0.pos,
_5qc_Lucene41_0.doc, _5hm_Lucene41_0.doc, _5p7.fnm, _5qt.si, _5p7_2.del,
_5qi_Lucene41_0.doc, _5r2_2.del, _5r3.fdx, _5o8.fdx, _5q1.fnm, _5nx.fdt,
_4qe.fnm, _5nx.fdx, _5q1.si, _5r3_Lucene41_0.pos, _5qa.fdt, _5r2.fdx,
_5o8.fdt, _5qa_Lucene41_0.pos, _5qc_1.del, _5qw.fnm, _5m7_Lucene41_0.doc,
_5qa.fdx, _5hm.fnm, _554.si, _5r4.si, _5qy_Lucene41_0.pos,
_5qr_Lucene41_0.tim, _5qr_Lucene41_0.tip, _5qy.fdx, _5nx_Lucene41_0.pos,
_5m7.fdt, _5kb_Lucene41_0.doc, _5m7.fdx, _554_Lucene41_0.tip, _5qc.fnm,
_5o8_Lucene41_0.tim, _5co_Lucene41_0.doc, _554_Lucene41_0.tim, _5qy.fdt, _
5qx.si, _5o8_Lucene41_0.tip, _5qu.fnm, _5nx_Lucene41_0.doc, _5q2.fnm,
_5o8_Lucene41_0.pos, _5qt_Lucene41_0.pos, _5qt.fnm, _5qh_Lucene41_0.tip,
_5qh_Lucene41_0.tim, _5q2_Lucene41_0.tip, _5q2_Lucene41_0.tim,
_5q0_Lucene41_0.doc, _5hm_c.del, _5r5_1.del, _5o8.fnm, _5qu_Lucene41_0.doc,
_5q0_Lucene41_0.tip, _5hm_Lucene41_0.tip, _5r5_Lucene41_0.pos, _5qo.fdx,
_5f5.fdt, _5q0_Lucene41_0.tim, _5qr.fdx, _5qu_Lucene41_0.tim, _5p7.si,
_5q1_1.del, _5of_1.del, _5qp.fdt, _5q0.fdx, _5q0.si, _5r4_Lucene41_0.doc,
_5of.fdx, _4qe_Lucene41_0.doc, _5qh.fnm, _5of.fdt, _5of_Lucene41_0.pos,
_5f5.fdx, _5m7_Lucene41_0.pos, _5r3.si, _5qw.fdt, _5qx_Lucene41_0.tim,
_5qx_Lucene41_0.tip, _5nx_2.del, _5qv_Lucene41_0.doc, _5qw.fdx, _5q0.fdt,
_5co_Lucene41_0.tim, _5qq_Lucene41_0.doc, _5qp.fdx, _5qu_Lucene41_0.tip,
_5qr.fdt, _5qi_Lucene41_0.tip, _5qp_Lucene41_0.pos, _5hm_Lucene41_0.tim,
_5qi_Lucene41_0.tim, _5qo.fdt, _5co_Lucene41_0.tip, _5qc_Lucene41_0.pos]
14:16:53,013 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
pendingCommit != null
14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
wrote segments file "segments_9u"
14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1" [34 segments ; isCommit = true]
14:16:53,015 INFO  [org.apache.solr.core.SolrCore]
(recoveryExecutor-7-thread-1) SolrDeletionPolicy.onCommit: commits: num=2
        commit{dir=NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9t,generation=353}
        commit{dir=NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9u,generation=354}
14:16:53,016 INFO  [org.apache.solr.core.SolrCore]
(recoveryExecutor-7-thread-1) newest commit generation = 354
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]:
deleteCommits: now decRef commit "segments_9t"
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"segments_9t"
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 1 msec to
checkpoint
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
took 3733.0 msec
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
done
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=0 deletes=false hasTickets:false
pendingChangesInFullFlush: false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
nrtIsCurrent: infoVersion matches: false; DW changes: false; BD changes:
false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: flush at
getReader
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
startFullFlush
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=0 deletes=false hasTickets:false
pendingChangesInFullFlush: false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
deletes during flush
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
applyDeletes: no deletes; skipping
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: prune
sis=segments_9u: _4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: return
reader version=16082 reader=StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
recoveryExecutor-7-thread-1 finishFullFlush success=true
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: getReader
took 9 msec
14:16:53,026 INFO  [org.apache.solr.search.SolrIndexSearcher]
(recoveryExecutor-7-thread-1) Opening Searcher@dbe17f9[search1_shard7_replica4]
main
14:16:53,120 INFO  [org.apache.solr.update.UpdateHandler]
(recoveryExecutor-7-thread-1) end_commit_flush

==> gc.20150126-135638.log <==
1214.654: [GC1214.654: [ParNew: 974509K->56784K(996800K), 0.0150450 secs]
1519683K->603424K(1995752K), 0.0152330 secs] [Times: user=0.14 sys=0.00,
real=0.02 secs]

==> server.log <==
14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 477 msec to merge postings [8708
docs]
14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 0 msec to merge doc values [8708
docs]
14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge codec=Lucene410
docCount=8708; merged segment has no vectors; no norms; no docValues; prox;
freqs
14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merged segment size=7.623 MB vs
estimate=8.906 MB
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: commitMerge: _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: commitMergeDeletes
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: no new deletes or field updates
since merge started
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IFD][Lucene Merge Thread #0]: now checkpoint
"_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
_5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
_5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1" [25 segments ; isCommit = false]
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IFD][Lucene Merge Thread #0]: 0 msec to checkpoint
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: after commitMerge:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
_5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
_5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]: findMerges: 25 segments
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
large]
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
large]
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r3(4.10.0):C26057
size=23.893 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r6(4.10.0):C8708
size=7.622 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5qq(4.10.0):C1997
size=1.781 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5q2(4.10.0):C779
size=1.554 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   allowedSegmentCount=31 vs
count=25 (eligible count=23) tooBigCount=2
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge time 625 msec for 8708 docs
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: done

==> gc.20150126-135638.log <==
1217.747: [GC1217.747: [ParNew: 942864K->33781K(996800K), 0.0409520 secs]
1489504K->580420K(1995752K), 0.0411310 secs] [Times: user=0.25 sys=0.00,
real=0.04 secs]

==> server.log <==
14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) QuerySenderListener sending requests to
Searcher@dbe17f9[search1_shard7_replica4]
main{StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)}
14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) QuerySenderListener done.
14:16:58,365 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) [search1_shard7_replica4] Registered new
searcher Searcher@dbe17f9[search1_shard7_replica4]
main{StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)}
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_4qe_1k.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5p7_1.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5nx_1.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5co_a.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5q0_6.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5hm_b.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5m7_b.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5kb_b.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5f5_1.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5qo_1.del"



After this point it becomes active.

On Sun, Jan 25, 2015 at 9:53 PM, Erick Erickson <er...@gmail.com>
wrote:

> Ah, OK. Whew! because I was wondering how you were running at _all_ if all
> the memory was allocated to the JVM ;)..
>
> What is your Zookeeper timeout? The original default was 15 seconds and
> this
> has caused problems like this. Here's the scenario:
> You send a bunch of docs at the server, and eventually you hit a
> stop-the-world
> GC that takes longer than the Zookeeper timeout. So ZK thinks the node is
> down
> and initiates recovery. Eventually, you hit this on all the replicas.
>
> Sometimes I've seen situations where the answer is giving a bit more memory
> to the JVM, say 2-4G in your case. The theory here (and this is a shot in
> the
> dark) that your peak JVM requirements are close to your 12G, so the garbage
> collection spends enormous amounts of time collecting a small bit of
> memory,
> runs for some fraction of a second and does it again. Adding more to the
> JVMs
> memory allows the parallel collections to work without so many
> stop-the-world
> GC pauses.
>
> So what I'd do is turn on GC logging (probably on the replicas) and look
> for
> very long GC pauses. Mark Miller put together a blog here:
> https://lucidworks.com/blog/garbage-collection-bootcamp-1-0/
>
> See the "getting a view into garbage collection". The smoking gun here
> is if you see full GC pauses that are longer than the ZK timeout.
>
> 90M docs in 4 hours across 10 shards is only 625/sec or so per shard. I've
> seen
> sustained indexing rates significantly above this, YMMV or course, a lot
> depends
> on the size of the docs.
>
> What version of Solr BTW? And when you say you fire a bunch of indexers,
> I'm
> assuming these are SolrJ clients and use CloudSolrServer?
>
> Best,
> Erick
>
>
> On Sun, Jan 25, 2015 at 4:10 PM, Vijay Sekhri <se...@gmail.com>
> wrote:
>
> > Thank you for the reply Eric.
> > I am sorry I had wrong information posted. I posted our DEV env
> > configuration by mistake.
> > After double checking our stress and Prod Beta env where we have found
> the
> > original issue, I found all the searchers have around 50 GB of RAM
> > available and two instances of JVM running (2 different ports). Both
> > instances have 12 GB allocated. The rest 26 GB is available for the OS.
> 1st
> >  instance on a host has search1 collection (live collection) and the 2nd
> > instance on the same host  has search2 collection (for full indexing ).
> >
> > There is plenty room for OS related tasks. Our issue is not in anyway
> > related to OS starving as shown from our dashboards.
> > We have been through
> >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > a lot of times but  we have two modes of operation
> > a)  1st collection (Live traffic) - heavy searches and medium indexing
> > b)  2nd collection (Not serving traffic) - very heavy indexing, no
> searches
> >
> > When our indexing finishes we swap the alias for these collection . So
> > essentially we need to have a configuration that can support both the use
> > cases together. We have tried a lot of different configuration options
> and
> > none of them seems to work. My suspicion is that solr cloud is unable to
> > keep up with the updates at the rate we are sending while it is trying to
> > be consistent with all the replicas.
> >
> >
> > On Sun, Jan 25, 2015 at 5:30 PM, Erick Erickson <erickerickson@gmail.com
> >
> > wrote:
> >
> > > Shawn directed you over here to the user list, but I see this note on
> > > SOLR-7030:
> > > "All our searchers have 12 GB of RAM available and have quad core
> > Intel(R)
> > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> > > jboss and solr in it . All 12 GB is available as heap for the java
> > > process..."
> > >
> > > So you have 12G physical memory and have allocated 12G to the Java
> > process?
> > > This is an anti-pattern. If that's
> > > the case, your operating system is being starved for memory, probably
> > > hitting a state where it spends all of its
> > > time in stop-the-world garbage collection, eventually it doesn't
> respond
> > to
> > > Zookeeper's ping so Zookeeper
> > > thinks the node is down and puts it into recovery. Where it spends a
> lot
> > of
> > > time doing... essentially nothing.
> > >
> > > About the hard and soft commits: I suspect these are entirely
> unrelated,
> > > but here's a blog on what they do, you
> > > should pick the configuration that supports your use case (i.e. how
> much
> > > latency can you stand between indexing
> > > and being able to search?).
> > >
> > >
> > >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > >
> > > Here's one very good reason you shouldn't starve your op system by
> > > allocating all the physical memory to the JVM:
> > >
> http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html
> > >
> > >
> > > But your biggest problem is that you have far too much of your physical
> > > memory allocated to the JVM. This
> > > will cause you endless problems, you just need more physical memory on
> > > those boxes. It's _possible_ you could
> > > get by with less memory for the JVM, counterintuitive as it seems try
> 8G
> > or
> > > maybe even 6G. At some point
> > > you'll hit OOM errors, but that'll give you a lower limit on what the
> JVM
> > > needs.
> > >
> > > Unless I've mis-interpreted what you've written, though, I doubt you'll
> > get
> > > stable with that much memory allocated
> > > to the JVM.
> > >
> > > Best,
> > > Erick
> > >
> > >
> > >
> > > On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <se...@gmail.com>
> > > wrote:
> > >
> > > > We have a cluster of solr cloud server with 10 shards and 4 replicas
> in
> > > > each shard in our stress environment. In our prod environment we will
> > > have
> > > > 10 shards and 15 replicas in each shard. Our current commit settings
> > are
> > > as
> > > > follows
> > > >
> > > > *    <autoSoftCommit>*
> > > > *        <maxDocs>500000</maxDocs>*
> > > > *        <maxTime>180000</maxTime>*
> > > > *    </autoSoftCommit>*
> > > > *    <autoCommit>*
> > > > *        <maxDocs>2000000</maxDocs>*
> > > > *        <maxTime>180000</maxTime>*
> > > > *        <openSearcher>false</openSearcher>*
> > > > *    </autoCommit>*
> > > >
> > > >
> > > > We indexed roughly 90 Million docs. We have two different ways to
> index
> > > > documents a) Full indexing. It takes 4 hours to index 90 Million docs
> > and
> > > > the rate of docs coming to the searcher is around 6000 per second b)
> > > > Incremental indexing. It takes an hour to indexed delta changes.
> > Roughly
> > > > there are 3 million changes and rate of docs coming to the searchers
> is
> > > > 2500
> > > > per second
> > > >
> > > > We have two collections search1 and search2. When we do full
> indexing ,
> > > we
> > > > do it in search2 collection while search1 is serving live traffic.
> > After
> > > it
> > > > finishes we swap the collection using aliases so that the search2
> > > > collection serves live traffic while search1 becomes available for
> next
> > > > full indexing run. When we do incremental indexing we do it in the
> > > search1
> > > > collection which is serving live traffic.
> > > >
> > > > All our searchers have 12 GB of RAM available and have quad core
> > Intel(R)
> > > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running
> i.e
> > > > jboss and solr in it . All 12 GB is available as heap for the java
> > > > process.  We have observed that the heap memory of the java process
> > > average
> > > > around 8 - 10 GB. All searchers have final index size of 9 GB. So in
> > > total
> > > > there are 9X10 (shards) =  90GB worth of index files.
> > > >
> > > >  We have observed the following issue when we trigger indexing . In
> > about
> > > > 10 minutes after we trigger indexing on 14 parallel hosts, the
> replicas
> > > > goes in to recovery mode. This happens to all the shards . In about
> 20
> > > > minutes more and more replicas start going into recovery mode. After
> > > about
> > > > half an hour all replicas except the leader are in recovery mode. We
> > > cannot
> > > > throttle the indexing load as that will increase our overall indexing
> > > time.
> > > > So to overcome this issue, we remove all the replicas before we
> trigger
> > > the
> > > > indexing and then add them back after the indexing finishes.
> > > >
> > > > We observe the same behavior of replicas going into recovery when we
> do
> > > > incremental indexing. We cannot remove replicas during our
> incremental
> > > > indexing because it is also serving live traffic. We tried to
> throttle
> > > our
> > > > indexing speed , however the cluster still goes into recovery .
> > > >
> > > > If we leave the cluster as it , when the indexing finishes , it
> > > eventually
> > > > recovers after a while. As it is serving live traffic we cannot have
> > > these
> > > > replicas go into recovery mode because it degrades the search
> > performance
> > > > also , our tests have shown.
> > > >
> > > > We have tried different commit settings like below
> > > >
> > > > a) No auto soft commit, no auto hard commit and a commit triggered at
> > the
> > > > end of indexing b) No auto soft commit, yes auto hard commit and a
> > commit
> > > > in the end of indexing
> > > > c) Yes auto soft commit , no auto hard commit
> > > > d) Yes auto soft commit , yes auto hard commit
> > > > e) Different frequency setting for commits for above. Please NOTE
> that
> > we
> > > > have tried 15 minute soft commit setting and 30 minutes hard commit
> > > > settings. Same time settings for both, 30 minute soft commit and an
> > hour
> > > > hard commit setting
> > > >
> > > > Unfortunately all the above yields the same behavior . The replicas
> > still
> > > > goes in recovery We have increased the zookeeper timeout from 30
> > seconds
> > > to
> > > > 5 minutes and the problem persists. Is there any setting that would
> fix
> > > > this issue ?
> > > >
> > > > --
> > > > *********************************************
> > > > Vijay Sekhri
> > > > *********************************************
> > > >
> > >
> >
> >
> >
> > --
> > *********************************************
> > Vijay Sekhri
> > *********************************************
> >
>



-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Erick Erickson <er...@gmail.com>.
Personally, I never really set maxDocs for autocommit, I just leave things
time-based. That said, your settings are so high that this shouldn't matter
in the least.

There's nothing in the log fragments you posted that's the proverbial
"smoking gun". There's
nothing here that tells me _why_ the node went into recovery in the first
place.

bq: Now we observed that replicas do not go in recovery that often as
before.

Hmmm, that at least seems to be pointing us in the direction of ZK not
being able
to see the server, marking it as down so it goes into recovery.

Why this is happening is still a mystery to me though. I'd expect the Solr
logs to show
"connection timeout", a stack trace or some such. _Some_ indication of why
the node
thinks it's out of sync.

Sorry, I'm a bit clueless here.

Erick



On Mon, Jan 26, 2015 at 1:34 PM, Vijay Sekhri <se...@gmail.com> wrote:

> Hi Erick,
> In solr.xml file I had zk timeout set to*  <int
> name="zkClientTimeout">${zkClientTimeout:450000}</int>*
> One thing that made a it a bit better now is the zk tick time and syncLimit
> settings. I set it to a higher value as below. This may not be advisable
> though.
>
> tickTime=30000
> initLimit=30
> syncLimit=20
>
> Now we observed that replicas do not go in recovery that often as before.
> In the whole cluster at a given time I would have a couple of replicas in
> recovery whereas earlier it were multiple replicas from every shard .
> On the wiki https://wiki.apache.org/solr/SolrCloud it says the "The
> maximum
> is 20 times the tickTime." in the FAQ so I decided to increase the tick
> time. Is this the correct approach ?
>
> One question I have is that if auto commit settings has anything to do with
> this or not ? Does it induce extra work for the searchers because of which
> this would happen? I have tried with following settings
> *  <autoSoftCommit>*
> *            <maxDocs>500000</maxDocs>*
> *            <maxTime>900000</maxTime>*
> *        </autoSoftCommit>*
>
> *        <autoCommit>*
> *            <maxDocs>200000</maxDocs>*
> *            <maxTime>30000</maxTime>*
> *            <openSearcher>false</openSearcher>*
> *        </autoCommit>*
>
> I have increased  the  heap size to 15GB for each JVM instance . I
> monitored during full indexing how the heap usage looks like and it never
> goes beyond 8 GB .  I don't see any Full GC happening at any point .
> I had some attached screenshots but they were marked as spam so not sending
> them again
>
>
>
> Our rate is a variable rate . It is not a sustained rate of 6000/second ,
> however there are intervals where it would reach that much and come down
> and grow again and come down.  So if I would take an average it would be
> 600/second only but that is not real rate at any given time.
> Version of solr cloud is 4.10.  All indexers are basically java programs
> running on different host using CloudSolrServer api.
> As I mentioned it is much better now than before , however not completely
> as expected . We would want none of them to go in recovery if really there
> is no need.
>
> I captured some logs before and after recovery
>
>  4:13:54,298 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
> New index installed. Updating index properties...
> index=index.20150126140904697
> 14:13:54,301 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
> removing old index directory
> NRTCachingDirectory(MMapDirectory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
> lockFactory=NativeFSLockFactory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417;
> maxCacheMB=48.0 maxMergeSizeMB=4.0)
> 14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
> (RecoveryThread) Creating new IndexWriter...
> 14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
> (RecoveryThread) Waiting until IndexWriter is unused...
> core=search1_shard7_replica4
> 14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
> (RecoveryThread) Rollback old IndexWriter... core=search1_shard7_replica4
> 14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: rollback
> 14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: all running merges have aborted
> 14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: rollback: done finish merges
> 14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [DW][RecoveryThread]: abort
> 14:13:54,303 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [DW][RecoveryThread]: done abort; abortedFiles=[]
> success=true
> 14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: rollback:
> infos=_4qe(4.10.0):C4312879/1370002:delGen=56
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
> _5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
> _5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
> _5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
> _5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
> _5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
> _5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
> _5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122
> 14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IFD][RecoveryThread]: now checkpoint
> "_4qe(4.10.0):C4312879/1370002:delGen=56
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
> _5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
> _5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
> _5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
> _5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
> _5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
> _5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
> _5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122" [30 segments ;
> isCommit = false]
> 14:13:54,307 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IFD][RecoveryThread]: 0 msec to checkpoint
> 14:13:54,323 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread) New
> index directory detected:
>
> old=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
>
> new=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
> 14:13:54,417 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IFD][RecoveryThread]: init: current segments file is
> "segments_9t";
> deletionPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper@592ddff6
> 14:13:54,420 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IFD][RecoveryThread]: init: load commit "segments_9t"
> 14:13:54,424 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread)
> SolrDeletionPolicy.onInit: commits: num=1
>         commit{dir=NRTCachingDirectory(MMapDirectory@
>
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
> ersion=4.10.0
> matchVersion=4.8.0
> analyzer=null
> ramBufferSizeMB=100.0
> maxBufferedDocs=-1
> maxBufferedDeleteTerms=-1
> mergedSegmentWarmer=null
> readerTermsIndexDivisor=1
> termIndexInterval=32
> delPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper
> commit=null
> openMode=APPEND
> similarity=com.shc.solrx.similarity.CustomSimilarity
> mergeScheduler=ConcurrentMergeScheduler: maxThreadCount=1, maxMergeCount=2,
> mergeThreadPriority=-1
> default WRITE_LOCK_TIMEOUT=1000
> writeLockTimeout=1000
> codec=Lucene410
> infoStream=org.apache.solr.update.LoggingInfoStream
> mergePolicy=[TieredMergePolicy: maxMergeAtOnce=10,
> maxMergeAtOnceExplicit=30, maxMergedSegmentMB=5120.0, floorSegmentMB=2.0,
> forceMergeDeletesPctAllowed=10.0, segmentsPerTier=10.0,
> maxCFSSegmentSizeMB=8.796093022207999E12, noCFSRatio=0.0
>
> indexerThreadPool=org.apache.lucene.index.DocumentsWriterPerThreadPool@693825c0
> readerPooling=false
> perThreadHardLimitMB=1945
> useCompoundFile=false
> checkIntegrityAtMerge=false
> writer=org.apache.lucene.util.SetOnce@65369637
>
> 14:13:54,425 INFO  [org.apache.solr.update.DefaultSolrCoreState]
> (RecoveryThread) New IndexWriter is ready to be used.
> 14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: flush at getReader
> 14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [DW][RecoveryThread]: startFullFlush
> 14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [DW][RecoveryThread]: anyChanges? numDocsInRam=0
> deletes=false hasTickets:false pendingChangesInFullFlush: false
> 14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [IW][RecoveryThread]: apply all deletes during flush
> 14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [BD][RecoveryThread]: applyDeletes: no deletes; skipping
> 14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
> (RecoveryThread) [BD][RecoveryThread]: prune sis=segments_9t:
> _4qe(4.10.0):C4312879/1370002
>
>
> lrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates
> }
> status=0 QTime=0
> 14:16:49,279 INFO  [org.apache.solr.core.SolrCore] (http-/10.235.47.41:8580
> -1)
> [search1_shard7_replica4] webapp=/solr path=/update params={distrib.from=
>
> http://solrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates
> }
> status=0 QTime=0
> 14:16:49,283 INFO  [org.apache.solr.update.UpdateHandler]
> (recoveryExecutor-7-thread-1) start
>
> commit{flags=2,optimize=false,openSearcher=true,waitSearcher=true,expungeDeletes=false,softCommit=false,prepareCommit=false}
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> start
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> enter lock
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> now prepare
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> prepareCommit: flush
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:   index
> before flush _4qe(4.10.0):C4312879/1370002:delGen=56
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
> _5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
> _5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
> _5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
> _5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
> _5r4(4.10.0):C23934/1:delGen=1
> 14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> startFullFlush
> 14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> anyChanges? numDocsInRam=24222 deletes=true hasTickets:false
> pendingChangesInFullFlush: false
> 14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWFC][recoveryExecutor-7-thread-1]:
> addFlushableState DocumentsWriterPerThread [pendingDeletes=gen=0 24221
> deleted terms (unique count=24220) bytesUsed=4455794, segment=_5r5,
> aborting=false, numDocsInRAM=24222, deleteQueue=DWDQ: [ generation: 1 ]]
> 14:16:49,322 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush
> postings as segment _5r5 numDocs=24222
>
> ==> gc.20150126-135638.log <==
> 1211.362: [GC1211.362: [ParNew: 966947K->88429K(996800K), 0.0191260 secs]
> 1499845K->633603K(1995752K), 0.0192710 secs] [Times: user=0.20 sys=0.00,
> real=0.02 secs]
>
> ==> server.log <==
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
> segment has 1 deleted docs
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
> segment has no vectors; no norms; no docValues; prox; freqs
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]:
> flushedFiles=[_5r5_Lucene41_0.pos, _5r5.fdx, _5r5.fnm, _5r5.fdt,
> _5r5_Lucene41_0.tim, _5r5_Lucene41_0.tip, _5r5_Lucene41_0.doc]
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed
> codec=Lucene410
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed:
> segment=_5r5 ramUsed=75.564 MB newFlushedSize(includes docstores)=19.546 MB
> docs/MB=1,239.201
> 14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush:
> write 1 deletes gen=-1
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> publishFlushedSegment seg-private updates=null
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> publishFlushedSegment
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: push
> deletes  24222 deleted terms (unique count=24221) bytesUsed=286752 delGen=8
> packetCount=4 totBytesUsed=1259648
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: publish
> sets newSegment delGen=9 seg=_5r5(4.10.0):C24222/1:delGen=1
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
> checkpoint "_4qe(4.10.0):C4312879/1370002:delGen=56
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
> _5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
> _5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
> _5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
> _5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
> _5r4(4.10.0):C23934/1:delGen=1 _5r5(4.10.0):C24222/1:delGen=1" [34 segments
> ; isCommit = false]
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
> checkpoint
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
> deletes during flush
> 14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
> applyDeletes: infos=[_4qe(4.10.0):C4312879/1370002:delGen=56,
> _554(4.10.0):C3995865/780418:delGen=23,
> _56u(4.10.0):C286775/11906:delGen=15, _5co(4.10.0):C871785/93841:delGen=10,
> _5m7(4.10.0):C122852/31675:delGen=11, _5hm(4.10.0):C457977/32535:delGen=11,
> _5q0(4.10.0):C13610/649:delGen=6, _5kb(4.10.0):C424868/19149:delGen=11,
> _5f5(4.10.0):C116528/42495:delGen=1, _5nx(4.10.0):C33236/20668:delGen=1,
> _5qm(4.10.0):C29770/1:delGen=1, _5o8(4.10.0):C27155/7531:delGen=1,
> _5of(4.10.0):C38545/5677:delGen=1, _5p7(4.10.0):C37457/648:delGen=1,
> _5qv(4.10.0):C3973, _5q1(4.10.0):C402/1:delGen=1, _5q2(4.10.0):C779,
> _5qa(4.10.0):C967, _5qc(4.10.0):C1828, _5qh(4.10.0):C1765,
> _5qi(4.10.0):C1241, _5qq(4.10.0):C1997, _5qr(4.10.0):C1468,
> _5qp(4.10.0):C1729, _5qo(4.10.0):C3456/1:delGen=1, _5qu(4.10.0):C27,
> _5qt(4.10.0):C30, _5qx(4.10.0):C638, _5qy(4.10.0):C1407, _5qw(4.10.0):C802,
> _5r2(4.10.0):C32769/1:delGen=1, _5r3(4.10.0):C26057,
> _5r4(4.10.0):C23934/1:delGen=1, _5r5(4.10.0):C24222/1:delGen=1]
> packetCount=4
> 14:16:50,402 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
> seg=_5r4(4.10.0):C23934/1:delGen=1 segGen=7
> 14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
> "_5r2_1.del"
> 14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
> checkpoint
> 14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
> checkpoint "_4qe(4.10.0):C4312879/1nts ; isCommit = false]
> 14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
> "_5r4_1.del"
> 14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
> checkpoint
> 14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> recoveryExecutor-7-thread-1 finishFullFlush success=true
> 14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> findMerges: 34 segments
> 14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
> large]
> 14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
> large]
> 14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
> 14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
> 14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5r3(4.10.0):C26057 size=23.893 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qq(4.10.0):C1997 size=1.781 MB [floored]
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5q2(4.10.0):C779 size=1.554 MB [floored]
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
> 14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qr(4.10.0):C1468 size=1.390 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qp(4.10.0):C1729/1:delGen=1 size=1.351 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qa(4.10.0):C967 size=1.235 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qi(4.10.0):C1241 size=1.146 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qy(4.10.0):C1407 size=1.050 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5q1(4.10.0):C402/1:delGen=1 size=0.954 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qw(4.10.0):C802/1:delGen=1 size=0.821 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qx(4.10.0):C638 size=0.818 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qt(4.10.0):C30 size=0.072 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> seg=_5qu(4.10.0):C27 size=0.063 MB [floored]
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> allowedSegmentCount=31 vs count=34 (eligible count=32) tooBigCount=2
> 14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> maybe=_5co(4.10.0):C871785/93995:delGen=11
> _5kb(4.10.0):C424868/49572:delGen=12 _5hm(4.10.0):C457977/83353:delGen=12
> _56u(4.10.0):C286775/11906:delGen=15 _5f5(4.10.0):C116528/43621:delGen=2
> _5m7(4.10.0):C122852/54010:delGen=12 _5p7(4.10.0):C37457/649:delGen=2
> _5of(4.10.0):C38545/5677:delGen=1 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 score=0.7313777596572674 skew=0.341
> nonDelRatio=0.852 tooLarge=false size=2506.203 MB
> .....
>
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> maybe=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
> _5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27 score=0.22289855634027114 skew=0.100 nonDelRatio=1.000
> tooLarge=false size=8.907 MB
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:   add
> merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
> _5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27 size=8.907 MB score=0.223 skew=0.100 nonDelRatio=1.000
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
> allowedSegmentCount=31 vs count=34 (eligible count=22) tooBigCount=2
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: add merge
> to pendingMerges: _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
> _5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
> _5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
> _5qt(4.10.0):C30 _5qu(4.10.0):C27 [total 1 pending]
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge merging= []
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qr(4.10.0):C1468
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qp(4.10.0):C1729/1:delGen=1
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qa(4.10.0):C967
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qi(4.10.0):C1241
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qy(4.10.0):C1407
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5q1(4.10.0):C402/1:delGen=1
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qw(4.10.0):C802/1:delGen=1
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qx(4.10.0):C638
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qt(4.10.0):C30
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> registerMerge info=_5qu(4.10.0):C27
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: now merge
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   index:
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
> consider merge _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
> _5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
> _5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
> _5qt(4.10.0):C30 _5qu(4.10.0):C27
> 14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
> launch new thread [Lucene Merge Thread #0]
> 14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: set
> priority of merge thread Lucene Merge Thread #0 to 6
> 14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   no more
> merges pending; now return
> 14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> startCommit(): start
> 14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> startCommit index=_4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1 changeCount=28
> 14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: start
> 14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [BD][Lucene Merge Thread #0]: applyDeletes: no deletes; skipping
> 14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [BD][Lucene Merge Thread #0]: prune sis=segments_9t:
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
> 14:16:52,788 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: merge seg=_5r6 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
> _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27
> 14:16:52,789 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: now merge
>   merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
> _5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27
>   index=_4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: merging _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
> _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qr(4.10.0):C1468 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qp(4.10.0):C1729/1:delGen=1
> delCount=1
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qa(4.10.0):C967 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qi(4.10.0):C1241 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qy(4.10.0):C1407 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5q1(4.10.0):C402/1:delGen=1
> delCount=1
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qw(4.10.0):C802/1:delGen=1
> delCount=1
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qx(4.10.0):C638 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qt(4.10.0):C30 no deletes
> 14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qu(4.10.0):C27 no deletes
> 14:16:52,792 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [SM][Lucene Merge Thread #0]: merge store matchedCount=10 vs 10
> 14:16:52,929 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [SM][Lucene Merge Thread #0]: 137 msec to merge stored fields
> [8708 docs]
> 14:16:53,012 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: done all
> syncs: [_5p7_Lucene41_0.pos, _5qr.si, _5kb_Lucene41_0.tim, _5q1.fdt,
> _5kb_Lucene41_0.tip, _5qp.fnm, _5o8_1.del, _5qa_Lucene41_0.doc, _5qq.fnm,
> _5qr.fnm, _5co_b.del, _5qh.fdx, _5qm.fdx, _5qh.fdt, _5kb_c.del, _5r5.fnm,
> _554.fdx, _5qm.fdt, _5r3_Lucene41_0.doc, _4qe.fdt, _56u_Lucene41_0.doc,
> _554.fdt, _5qi.fdt, _5q2_Lucene41_0.doc, _4qe.fdx, _5qa.si, _5qi.fdx, _
> 5qy.si, _56u.fdx, _5qv_Lucene41_0.tip, _5qh.si, _5qw_Lucene41_0.pos,
> _5hm_Lucene41_0.pos, _5nx.si, _5co_Lucene41_0.pos, _5q1.fdx, _5qt.fdx,
> _5qt.fdt, _5r5.si, _5co.fdt, _5nx.fnm, _5qv_Lucene41_0.tim, _5co.fdx,
> _5qh_Lucene41_0.doc, _5kb.fdx, _5qh_1.del, _5qm_Lucene41_0.tim, _56u.fdt,
> _5r4_2.del, _5r3_Lucene41_0.tip, _5hm.fdt, _5qv_1.del, _5qm_Lucene41_0.tip,
> _5qr_Lucene41_0.doc, _5kb.fdt, _5of_Lucene41_0.tip, _5hm.fdx,
> _5r3_Lucene41_0.tim, _5of_Lucene41_0.tim, _56u_f.del, _5qy_Lucene41_0.doc,
> _5r4_Lucene41_0.pos, _5qq.si, _5qt_Lucene41_0.doc, _5qq_Lucene41_0.pos,
> _5qa.fnm, _5qv.si, _5r2.fnm, _5qo_2.del, _5p7.fdt, _5r3.fnm,
> _5p7_Lucene41_0.doc, _5r4.fdt, _5qo.si, _5p7.fdx, _5qa_Lucene41_0.tim, _
> 5qp.si, _5kb.si, _5r4.fdx, _5qp_Lucene41_0.tip, _4qe.si,
> _5qp_Lucene41_0.tim, _5qa_Lucene41_0.tip, _4qe_1l.del, _5qp_1.del,
> _5qw_1.del, _5qm_2.del, _5m7_Lucene41_0.tim, _5qw.si,
> _5m7_Lucene41_0.tip, _
> 56u.si, _5of_Lucene41_0.doc, _5of.si, _5of.fnm, _4qe_Lucene41_0.tip,
> _5qh_Lucene41_0.pos, _5qi.si, _4qe_Lucene41_0.tim, _5qc.fdx,
> _5r2_Lucene41_0.pos, _5qm.fnm, _5qc.fdt, _5qx_Lucene41_0.pos,
> _5qo_Lucene41_0.doc, _5q0.fnm, _5qv.fdx, _5qi_Lucene41_0.pos,
> _5p7_Lucene41_0.tim, _5qv.fdt, _5f5.si, _5p7_Lucene41_0.tip, _5q2.fdt, _
> 5o8.si, _5qm_Lucene41_0.doc, _5qc_Lucene41_0.tim, _5f5_Lucene41_0.tim,
> _5f5_Lucene41_0.tip, _5qc_Lucene41_0.tip, _5qx.fdx, _554_Lucene41_0.doc,
> _5q2_Lucene41_0.pos, _5r4_Lucene41_0.tim, _5qx.fdt, _5q1_Lucene41_0.pos,
> _5qp_Lucene41_0.doc, _5qm.si, _56u.fnm, _5q0_Lucene41_0.pos,
> _5r4_Lucene41_0.tip, _5qq.fdx, _5qr_Lucene41_0.pos, _56u_Lucene41_0.pos,
> _5r2_Lucene41_0.doc, _5qw_Lucene41_0.doc, _5r5_Lucene41_0.tim,
> _5r5_Lucene41_0.tip, _554.fnm, _5qi.fnm, _56u_Lucene41_0.tim,
> _5qo_Lucene41_0.pos, _56u_Lucene41_0.tip, _5f5.fnm, _5qq.fdt, _5q0_7.del,
> _5m7.fnm, _5qo.fnm, _5qu.si, _5q2.fdx, _5qt_Lucene41_0.tip,
> _5qw_Lucene41_0.tip, _5qt_Lucene41_0.tim, _5qc.si, _5hm.si, _5m7.si, _
> 5r2.si,
> _5f5_Lucene41_0.pos, _5qy_Lucene41_0.tip, _5nx_Lucene41_0.tim,
> _5r2_Lucene41_0.tip, _5qm_Lucene41_0.pos, _5qq_Lucene41_0.tim, _5co.si,
> _5r2.fdt, _5r2_Lucene41_0.tim, _5f5_2.del, _5qq_Lucene41_0.tip,
> _5qy_Lucene41_0.tim, _5nx_Lucene41_0.tip, _5o8_Lucene41_0.doc, _5r3.fdt,
> _5qv_Lucene41_0.pos, _5q1_Lucene41_0.doc, _5qo_Lucene41_0.tip, _5qx.fnm,
> _5qo_Lucene41_0.tim, _5co.fnm, _5qu.fdt, _5qu.fdx, _5r5.fdx, _554_n.del,
> _4qe_Lucene41_0.pos, _5q1_Lucene41_0.tip, _5f5_Lucene41_0.doc, _5r5.fdt,
> _5qw_Lucene41_0.tim, _5m7_c.del, _5qv.fnm, _5q1_Lucene41_0.tim, _5kb.fnm, _
> 5q2.si, _5qy.fnm, _5kb_Lucene41_0.pos, _5qx_Lucene41_0.doc, _5r4.fnm,
> _5qu_Lucene41_0.pos, _5r5_Lucene41_0.doc, _554_Lucene41_0.pos,
> _5qc_Lucene41_0.doc, _5hm_Lucene41_0.doc, _5p7.fnm, _5qt.si, _5p7_2.del,
> _5qi_Lucene41_0.doc, _5r2_2.del, _5r3.fdx, _5o8.fdx, _5q1.fnm, _5nx.fdt,
> _4qe.fnm, _5nx.fdx, _5q1.si, _5r3_Lucene41_0.pos, _5qa.fdt, _5r2.fdx,
> _5o8.fdt, _5qa_Lucene41_0.pos, _5qc_1.del, _5qw.fnm, _5m7_Lucene41_0.doc,
> _5qa.fdx, _5hm.fnm, _554.si, _5r4.si, _5qy_Lucene41_0.pos,
> _5qr_Lucene41_0.tim, _5qr_Lucene41_0.tip, _5qy.fdx, _5nx_Lucene41_0.pos,
> _5m7.fdt, _5kb_Lucene41_0.doc, _5m7.fdx, _554_Lucene41_0.tip, _5qc.fnm,
> _5o8_Lucene41_0.tim, _5co_Lucene41_0.doc, _554_Lucene41_0.tim, _5qy.fdt, _
> 5qx.si, _5o8_Lucene41_0.tip, _5qu.fnm, _5nx_Lucene41_0.doc, _5q2.fnm,
> _5o8_Lucene41_0.pos, _5qt_Lucene41_0.pos, _5qt.fnm, _5qh_Lucene41_0.tip,
> _5qh_Lucene41_0.tim, _5q2_Lucene41_0.tip, _5q2_Lucene41_0.tim,
> _5q0_Lucene41_0.doc, _5hm_c.del, _5r5_1.del, _5o8.fnm, _5qu_Lucene41_0.doc,
> _5q0_Lucene41_0.tip, _5hm_Lucene41_0.tip, _5r5_Lucene41_0.pos, _5qo.fdx,
> _5f5.fdt, _5q0_Lucene41_0.tim, _5qr.fdx, _5qu_Lucene41_0.tim, _5p7.si,
> _5q1_1.del, _5of_1.del, _5qp.fdt, _5q0.fdx, _5q0.si, _5r4_Lucene41_0.doc,
> _5of.fdx, _4qe_Lucene41_0.doc, _5qh.fnm, _5of.fdt, _5of_Lucene41_0.pos,
> _5f5.fdx, _5m7_Lucene41_0.pos, _5r3.si, _5qw.fdt, _5qx_Lucene41_0.tim,
> _5qx_Lucene41_0.tip, _5nx_2.del, _5qv_Lucene41_0.doc, _5qw.fdx, _5q0.fdt,
> _5co_Lucene41_0.tim, _5qq_Lucene41_0.doc, _5qp.fdx, _5qu_Lucene41_0.tip,
> _5qr.fdt, _5qi_Lucene41_0.tip, _5qp_Lucene41_0.pos, _5hm_Lucene41_0.tim,
> _5qi_Lucene41_0.tim, _5qo.fdt, _5co_Lucene41_0.tip, _5qc_Lucene41_0.pos]
> 14:16:53,013 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> pendingCommit != null
> 14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> wrote segments file "segments_9u"
> 14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
> checkpoint "_4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1" [34 segments ; isCommit = true]
> 14:16:53,015 INFO  [org.apache.solr.core.SolrCore]
> (recoveryExecutor-7-thread-1) SolrDeletionPolicy.onCommit: commits: num=2
>         commit{dir=NRTCachingDirectory(MMapDirectory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
> lockFactory=NativeFSLockFactory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
> maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9t,generation=353}
>         commit{dir=NRTCachingDirectory(MMapDirectory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
> lockFactory=NativeFSLockFactory@
> /opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
> maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9u,generation=354}
> 14:16:53,016 INFO  [org.apache.solr.core.SolrCore]
> (recoveryExecutor-7-thread-1) newest commit generation = 354
> 14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]:
> deleteCommits: now decRef commit "segments_9t"
> 14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
> "segments_9t"
> 14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 1 msec to
> checkpoint
> 14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> took 3733.0 msec
> 14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
> done
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> anyChanges? numDocsInRam=0 deletes=false hasTickets:false
> pendingChangesInFullFlush: false
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
> nrtIsCurrent: infoVersion matches: false; DW changes: false; BD changes:
> false
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: flush at
> getReader
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> startFullFlush
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> anyChanges? numDocsInRam=0 deletes=false hasTickets:false
> pendingChangesInFullFlush: false
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
> deletes during flush
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
> applyDeletes: no deletes; skipping
> 14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: prune
> sis=segments_9u: _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
> 14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: return
> reader version=16082 reader=StandardDirectoryReader(segments_9u:16082:nrt
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1)
> 14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
> recoveryExecutor-7-thread-1 finishFullFlush success=true
> 14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
> (recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: getReader
> took 9 msec
> 14:16:53,026 INFO  [org.apache.solr.search.SolrIndexSearcher]
> (recoveryExecutor-7-thread-1) Opening Searcher@dbe17f9
> [search1_shard7_replica4]
> main
> 14:16:53,120 INFO  [org.apache.solr.update.UpdateHandler]
> (recoveryExecutor-7-thread-1) end_commit_flush
>
> ==> gc.20150126-135638.log <==
> 1214.654: [GC1214.654: [ParNew: 974509K->56784K(996800K), 0.0150450 secs]
> 1519683K->603424K(1995752K), 0.0152330 secs] [Times: user=0.14 sys=0.00,
> real=0.02 secs]
>
> ==> server.log <==
> 14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [SM][Lucene Merge Thread #0]: 477 msec to merge postings [8708
> docs]
> 14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [SM][Lucene Merge Thread #0]: 0 msec to merge doc values [8708
> docs]
> 14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: merge codec=Lucene410
> docCount=8708; merged segment has no vectors; no norms; no docValues; prox;
> freqs
> 14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: merged segment size=7.623 MB vs
> estimate=8.906 MB
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: commitMerge: _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
> _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27 index=_4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: commitMergeDeletes
> _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
> _5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
> _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
> _5qu(4.10.0):C27
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: no new deletes or field updates
> since merge started
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IFD][Lucene Merge Thread #0]: now checkpoint
> "_4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
> _5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
> _5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1" [25 segments ; isCommit = false]
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IFD][Lucene Merge Thread #0]: 0 msec to checkpoint
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: after commitMerge:
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
> _5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
> _5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1
> 14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]: findMerges: 25 segments
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
> large]
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
> large]
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r3(4.10.0):C26057
> size=23.893 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
> 14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r6(4.10.0):C8708
> size=7.622 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5qq(4.10.0):C1997
> size=1.781 MB [floored]
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5q2(4.10.0):C779
> size=1.554 MB [floored]
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:
> seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [TMP][Lucene Merge Thread #0]:   allowedSegmentCount=31 vs
> count=25 (eligible count=23) tooBigCount=2
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [IW][Lucene Merge Thread #0]: merge time 625 msec for 8708 docs
> 14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
> Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: done
>
> ==> gc.20150126-135638.log <==
> 1217.747: [GC1217.747: [ParNew: 942864K->33781K(996800K), 0.0409520 secs]
> 1489504K->580420K(1995752K), 0.0411310 secs] [Times: user=0.25 sys=0.00,
> real=0.04 secs]
>
> ==> server.log <==
> 14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
> (searcherExecutor-6-thread-1) QuerySenderListener sending requests to
> Searcher@dbe17f9[search1_shard7_replica4]
> main{StandardDirectoryReader(segments_9u:16082:nrt
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1)}
> 14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
> (searcherExecutor-6-thread-1) QuerySenderListener done.
> 14:16:58,365 INFO  [org.apache.solr.core.SolrCore]
> (searcherExecutor-6-thread-1) [search1_shard7_replica4] Registered new
> searcher Searcher@dbe17f9[search1_shard7_replica4]
> main{StandardDirectoryReader(segments_9u:16082:nrt
> _4qe(4.10.0):C4312879/1370113:delGen=57
> _554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
> _5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
> _5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
> _5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
> _5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
> _5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
> _5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
> _5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
> _5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
> _5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
> _5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
> _5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
> _5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
> _5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
> _5r5(4.10.0):C24222/1:delGen=1)}
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
> "_4qe_1k.del"
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
> "_5p7_1.del"
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
> "_5nx_1.del"
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
> "_5co_a.del"
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
> "_5q0_6.del"
> 14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
> (searcherExecutor-6-thread-1) [
>
> On Sun, Jan 25, 2015 at 9:53 PM, Erick Erickson <er...@gmail.com>
> wrote:
>
> > Ah, OK. Whew! because I was wondering how you were running at _all_ if
> all
> > the memory was allocated to the JVM ;)..
> >
> > What is your Zookeeper timeout? The original default was 15 seconds and
> > this
> > has caused problems like this. Here's the scenario:
> > You send a bunch of docs at the server, and eventually you hit a
> > stop-the-world
> > GC that takes longer than the Zookeeper timeout. So ZK thinks the node is
> > down
> > and initiates recovery. Eventually, you hit this on all the replicas.
> >
> > Sometimes I've seen situations where the answer is giving a bit more
> memory
> > to the JVM, say 2-4G in your case. The theory here (and this is a shot in
> > the
> > dark) that your peak JVM requirements are close to your 12G, so the
> garbage
> > collection spends enormous amounts of time collecting a small bit of
> > memory,
> > runs for some fraction of a second and does it again. Adding more to the
> > JVMs
> > memory allows the parallel collections to work without so many
> > stop-the-world
> > GC pauses.
> >
> > So what I'd do is turn on GC logging (probably on the replicas) and look
> > for
> > very long GC pauses. Mark Miller put together a blog here:
> > https://lucidworks.com/blog/garbage-collection-bootcamp-1-0/
> >
> > See the "getting a view into garbage collection". The smoking gun here
> > is if you see full GC pauses that are longer than the ZK timeout.
> >
> > 90M docs in 4 hours across 10 shards is only 625/sec or so per shard.
> I've
> > seen
> > sustained indexing rates significantly above this, YMMV or course, a lot
> > depends
> > on the size of the docs.
> >
> > What version of Solr BTW? And when you say you fire a bunch of indexers,
> > I'm
> > assuming these are SolrJ clients and use CloudSolrServer?
> >
> > Best,
> > Erick
> >
> >
> > On Sun, Jan 25, 2015 at 4:10 PM, Vijay Sekhri <se...@gmail.com>
> > wrote:
> >
> > > Thank you for the reply Eric.
> > > I am sorry I had wrong information posted. I posted our DEV env
> > > configuration by mistake.
> > > After double checking our stress and Prod Beta env where we have found
> > the
> > > original issue, I found all the searchers have around 50 GB of RAM
> > > available and two instances of JVM running (2 different ports). Both
> > > instances have 12 GB allocated. The rest 26 GB is available for the OS.
> > 1st
> > >  instance on a host has search1 collection (live collection) and the
> 2nd
> > > instance on the same host  has search2 collection (for full indexing ).
> > >
> > > There is plenty room for OS related tasks. Our issue is not in anyway
> > > related to OS starving as shown from our dashboards.
> > > We have been through
> > >
> > >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > > a lot of times but  we have two modes of operation
> > > a)  1st collection (Live traffic) - heavy searches and medium indexing
> > > b)  2nd collection (Not serving traffic) - very heavy indexing, no
> > searches
> > >
> > > When our indexing finishes we swap the alias for these collection . So
> > > essentially we need to have a configuration that can support both the
> use
> > > cases together. We have tried a lot of different configuration options
> > and
> > > none of them seems to work. My suspicion is that solr cloud is unable
> to
> > > keep up with the updates at the rate we are sending while it is trying
> to
> > > be consistent with all the replicas.
> > >
> > >
> > > On Sun, Jan 25, 2015 at 5:30 PM, Erick Erickson <
> erickerickson@gmail.com
> > >
> > > wrote:
> > >
> > > > Shawn directed you over here to the user list, but I see this note on
> > > > SOLR-7030:
> > > > "All our searchers have 12 GB of RAM available and have quad core
> > > Intel(R)
> > > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running
> i.e
> > > > jboss and solr in it . All 12 GB is available as heap for the java
> > > > process..."
> > > >
> > > > So you have 12G physical memory and have allocated 12G to the Java
> > > process?
> > > > This is an anti-pattern. If that's
> > > > the case, your operating system is being starved for memory, probably
> > > > hitting a state where it spends all of its
> > > > time in stop-the-world garbage collection, eventually it doesn't
> > respond
> > > to
> > > > Zookeeper's ping so Zookeeper
> > > > thinks the node is down and puts it into recovery. Where it spends a
> > lot
> > > of
> > > > time doing... essentially nothing.
> > > >
> > > > About the hard and soft commits: I suspect these are entirely
> > unrelated,
> > > > but here's a blog on what they do, you
> > > > should pick the configuration that supports your use case (i.e. how
> > much
> > > > latency can you stand between indexing
> > > > and being able to search?).
> > > >
> > > >
> > > >
> > >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > > >
> > > > Here's one very good reason you shouldn't starve your op system by
> > > > allocating all the physical memory to the JVM:
> > > >
> > http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html
> > > >
> > > >
> > > > But your biggest problem is that you have far too much of your
> physical
> > > > memory allocated to the JVM. This
> > > > will cause you endless problems, you just need more physical memory
> on
> > > > those boxes. It's _possible_ you could
> > > > get by with less memory for the JVM, counterintuitive as it seems try
> > 8G
> > > or
> > > > maybe even 6G. At some point
> > > > you'll hit OOM errors, but that'll give you a lower limit on what the
> > JVM
> > > > needs.
> > > >
> > > > Unless I've mis-interpreted what you've written, though, I doubt
> you'll
> > > get
> > > > stable with that much memory allocated
> > > > to the JVM.
> > > >
> > > > Best,
> > > > Erick
> > > >
> > > >
> > > >
> > > > On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <sekhrivijay@gmail.com
> >
> > > > wrote:
> > > >
> > > > > We have a cluster of solr cloud server with 10 shards and 4
> replicas
> > in
> > > > > each shard in our stress environment. In our prod environment we
> will
> > > > have
> > > > > 10 shards and 15 replicas in each shard. Our current commit
> settings
> > > are
> > > > as
> > > > > follows
> > > > >
> > > > > *    <autoSoftCommit>*
> > > > > *        <maxDocs>500000</maxDocs>*
> > > > > *        <maxTime>180000</maxTime>*
> > > > > *    </autoSoftCommit>*
> > > > > *    <autoCommit>*
> > > > > *        <maxDocs>2000000</maxDocs>*
> > > > > *        <maxTime>180000</maxTime>*
> > > > > *        <openSearcher>false</openSearcher>*
> > > > > *    </autoCommit>*
> > > > >
> > > > >
> > > > > We indexed roughly 90 Million docs. We have two different ways to
> > index
> > > > > documents a) Full indexing. It takes 4 hours to index 90 Million
> docs
> > > and
> > > > > the rate of docs coming to the searcher is around 6000 per second
> b)
> > > > > Incremental indexing. It takes an hour to indexed delta changes.
> > > Roughly
> > > > > there are 3 million changes and rate of docs coming to the
> searchers
> > is
> > > > > 2500
> > > > > per second
> > > > >
> > > > > We have two collections search1 and search2. When we do full
> > indexing ,
> > > > we
> > > > > do it in search2 collection while search1 is serving live traffic.
> > > After
> > > > it
> > > > > finishes we swap the collection using aliases so that the search2
> > > > > collection serves live traffic while search1 becomes available for
> > next
> > > > > full indexing run. When we do incremental indexing we do it in the
> > > > search1
> > > > > collection which is serving live traffic.
> > > > >
> > > > > All our searchers have 12 GB of RAM available and have quad core
> > > Intel(R)
> > > > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running
> > i.e
> > > > > jboss and solr in it . All 12 GB is available as heap for the java
> > > > > process.  We have observed that the heap memory of the java process
> > > > average
> > > > > around 8 - 10 GB. All searchers have final index size of 9 GB. So
> in
> > > > total
> > > > > there are 9X10 (shards) =  90GB worth of index files.
> > > > >
> > > > >  We have observed the following issue when we trigger indexing . In
> > > about
> > > > > 10 minutes after we trigger indexing on 14 parallel hosts, the
> > replicas
> > > > > goes in to recovery mode. This happens to all the shards . In about
> > 20
> > > > > minutes more and more replicas start going into recovery mode.
> After
> > > > about
> > > > > half an hour all replicas except the leader are in recovery mode.
> We
> > > > cannot
> > > > > throttle the indexing load as that will increase our overall
> indexing
> > > > time.
> > > > > So to overcome this issue, we remove all the replicas before we
> > trigger
> > > > the
> > > > > indexing and then add them back after the indexing finishes.
> > > > >
> > > > > We observe the same behavior of replicas going into recovery when
> we
> > do
> > > > > incremental indexing. We cannot remove replicas during our
> > incremental
> > > > > indexing because it is also serving live traffic. We tried to
> > throttle
> > > > our
> > > > > indexing speed , however the cluster still goes into recovery .
> > > > >
> > > > > If we leave the cluster as it , when the indexing finishes , it
> > > > eventually
> > > > > recovers after a while. As it is serving live traffic we cannot
> have
> > > > these
> > > > > replicas go into recovery mode because it degrades the search
> > > performance
> > > > > also , our tests have shown.
> > > > >
> > > > > We have tried different commit settings like below
> > > > >
> > > > > a) No auto soft commit, no auto hard commit and a commit triggered
> at
> > > the
> > > > > end of indexing b) No auto soft commit, yes auto hard commit and a
> > > commit
> > > > > in the end of indexing
> > > > > c) Yes auto soft commit , no auto hard commit
> > > > > d) Yes auto soft commit , yes auto hard commit
> > > > > e) Different frequency setting for commits for above. Please NOTE
> > that
> > > we
> > > > > have tried 15 minute soft commit setting and 30 minutes hard commit
> > > > > settings. Same time settings for both, 30 minute soft commit and an
> > > hour
> > > > > hard commit setting
> > > > >
> > > > > Unfortunately all the above yields the same behavior . The replicas
> > > still
> > > > > goes in recovery We have increased the zookeeper timeout from 30
> > > seconds
> > > > to
> > > > > 5 minutes and the problem persists. Is there any setting that would
> > fix
> > > > > this issue ?
> > > > >
> > > > > --
> > > > > *********************************************
> > > > > Vijay Sekhri
> > > > > *********************************************
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > *********************************************
> > > Vijay Sekhri
> > > *********************************************
> > >
> >
>
>
>
> --
> *********************************************
> Vijay Sekhri
> *********************************************
>

Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Hi Erick,
In solr.xml file I had zk timeout set to*  <int
name="zkClientTimeout">${zkClientTimeout:450000}</int>*
One thing that made a it a bit better now is the zk tick time and syncLimit
settings. I set it to a higher value as below. This may not be advisable
though.

tickTime=30000
initLimit=30
syncLimit=20

Now we observed that replicas do not go in recovery that often as before.
In the whole cluster at a given time I would have a couple of replicas in
recovery whereas earlier it were multiple replicas from every shard .
On the wiki https://wiki.apache.org/solr/SolrCloud it says the "The maximum
is 20 times the tickTime." in the FAQ so I decided to increase the tick
time. Is this the correct approach ?

One question I have is that if auto commit settings has anything to do with
this or not ? Does it induce extra work for the searchers because of which
this would happen? I have tried with following settings
*  <autoSoftCommit>*
*            <maxDocs>500000</maxDocs>*
*            <maxTime>900000</maxTime>*
*        </autoSoftCommit>*

*        <autoCommit>*
*            <maxDocs>200000</maxDocs>*
*            <maxTime>30000</maxTime>*
*            <openSearcher>false</openSearcher>*
*        </autoCommit>*

I have increased  the  heap size to 15GB for each JVM instance . I
monitored during full indexing how the heap usage looks like and it never
goes beyond 8 GB .  I don't see any Full GC happening at any point .
I had some attached screenshots but they were marked as spam so not sending
them again



Our rate is a variable rate . It is not a sustained rate of 6000/second ,
however there are intervals where it would reach that much and come down
and grow again and come down.  So if I would take an average it would be
600/second only but that is not real rate at any given time.
Version of solr cloud is 4.10.  All indexers are basically java programs
running on different host using CloudSolrServer api.
As I mentioned it is much better now than before , however not completely
as expected . We would want none of them to go in recovery if really there
is no need.

I captured some logs before and after recovery

 4:13:54,298 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
New index installed. Updating index properties...
index=index.20150126140904697
14:13:54,301 INFO  [org.apache.solr.handler.SnapPuller] (RecoveryThread)
removing old index directory
NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417;
maxCacheMB=48.0 maxMergeSizeMB=4.0)
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Creating new IndexWriter...
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Waiting until IndexWriter is unused...
core=search1_shard7_replica4
14:13:54,302 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) Rollback old IndexWriter... core=search1_shard7_replica4
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: all running merges have aborted
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback: done finish merges
14:13:54,302 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: abort
14:13:54,303 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: done abort; abortedFiles=[]
success=true
14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: rollback:
infos=_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
_5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
_5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
_5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
_5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
_5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
_5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122
14:13:54,306 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: now checkpoint
"_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31645:delGen=11
_5hm(4.10.0):C457977/32465:delGen=11 _5q2(4.10.0):C13189/649:delGen=6
_5kb(4.10.0):C424868/19148:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5ql(4.10.0):C25924/2:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5r5(4.10.0):C4260 _5qv(4.10.0):C1750
_5qi(4.10.0):C842 _5qp(4.10.0):C2247 _5qm(4.10.0):C2214 _5qo(4.10.0):C1785
_5qn(4.10.0):C1962 _5qu(4.10.0):C2390 _5qy(4.10.0):C2129 _5qx(4.10.0):C2192
_5qw(4.10.0):C2157/1:delGen=1 _5r6(4.10.0):C159 _5r4(4.10.0):C742
_5r8(4.10.0):C334 _5r7(4.10.0):C390 _5r3(4.10.0):C1122" [30 segments ;
isCommit = false]
14:13:54,307 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: 0 msec to checkpoint
14:13:54,323 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread) New
index directory detected:
old=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126134945417
new=/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
14:13:54,417 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: init: current segments file is
"segments_9t";
deletionPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper@592ddff6
14:13:54,420 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IFD][RecoveryThread]: init: load commit "segments_9t"
14:13:54,424 INFO  [org.apache.solr.core.SolrCore] (RecoveryThread)
SolrDeletionPolicy.onInit: commits: num=1
        commit{dir=NRTCachingDirectory(MMapDirectory@
/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
ersion=4.10.0
matchVersion=4.8.0
analyzer=null
ramBufferSizeMB=100.0
maxBufferedDocs=-1
maxBufferedDeleteTerms=-1
mergedSegmentWarmer=null
readerTermsIndexDivisor=1
termIndexInterval=32
delPolicy=org.apache.solr.core.IndexDeletionPolicyWrapper
commit=null
openMode=APPEND
similarity=com.shc.solrx.similarity.CustomSimilarity
mergeScheduler=ConcurrentMergeScheduler: maxThreadCount=1, maxMergeCount=2,
mergeThreadPriority=-1
default WRITE_LOCK_TIMEOUT=1000
writeLockTimeout=1000
codec=Lucene410
infoStream=org.apache.solr.update.LoggingInfoStream
mergePolicy=[TieredMergePolicy: maxMergeAtOnce=10,
maxMergeAtOnceExplicit=30, maxMergedSegmentMB=5120.0, floorSegmentMB=2.0,
forceMergeDeletesPctAllowed=10.0, segmentsPerTier=10.0,
maxCFSSegmentSizeMB=8.796093022207999E12, noCFSRatio=0.0
indexerThreadPool=org.apache.lucene.index.DocumentsWriterPerThreadPool@693825c0
readerPooling=false
perThreadHardLimitMB=1945
useCompoundFile=false
checkIntegrityAtMerge=false
writer=org.apache.lucene.util.SetOnce@65369637

14:13:54,425 INFO  [org.apache.solr.update.DefaultSolrCoreState]
(RecoveryThread) New IndexWriter is ready to be used.
14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: flush at getReader
14:13:54,425 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: startFullFlush
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [DW][RecoveryThread]: anyChanges? numDocsInRam=0
deletes=false hasTickets:false pendingChangesInFullFlush: false
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [IW][RecoveryThread]: apply all deletes during flush
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [BD][RecoveryThread]: applyDeletes: no deletes; skipping
14:13:54,426 INFO  [org.apache.solr.update.LoggingInfoStream]
(RecoveryThread) [BD][RecoveryThread]: prune sis=segments_9t:
_4qe(4.10.0):C4312879/1370002

lrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates}
status=0 QTime=0
14:16:49,279 INFO  [org.apache.solr.core.SolrCore] (http-/10.235.47.41:8580-1)
[search1_shard7_replica4] webapp=/solr path=/update params={distrib.from=
http://solrx334p.qa.ch3.s.com:8680/solr/search1_shard7_replica11/&update.distrib=FROMLEADER&wt=javabin&version=2&update.chain=script&update.chain=removeDuplicates}
status=0 QTime=0
14:16:49,283 INFO  [org.apache.solr.update.UpdateHandler]
(recoveryExecutor-7-thread-1) start
commit{flags=2,optimize=false,openSearcher=true,waitSearcher=true,expungeDeletes=false,softCommit=false,prepareCommit=false}
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
start
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
enter lock
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
now prepare
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
prepareCommit: flush
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:   index
before flush _4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
_5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
_5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
_5r4(4.10.0):C23934/1:delGen=1
14:16:49,283 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
startFullFlush
14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=24222 deletes=true hasTickets:false
pendingChangesInFullFlush: false
14:16:49,284 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWFC][recoveryExecutor-7-thread-1]:
addFlushableState DocumentsWriterPerThread [pendingDeletes=gen=0 24221
deleted terms (unique count=24220) bytesUsed=4455794, segment=_5r5,
aborting=false, numDocsInRAM=24222, deleteQueue=DWDQ: [ generation: 1 ]]
14:16:49,322 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush
postings as segment _5r5 numDocs=24222

==> gc.20150126-135638.log <==
1211.362: [GC1211.362: [ParNew: 966947K->88429K(996800K), 0.0191260 secs]
1499845K->633603K(1995752K), 0.0192710 secs] [Times: user=0.20 sys=0.00,
real=0.02 secs]

==> server.log <==
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
segment has 1 deleted docs
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: new
segment has no vectors; no norms; no docValues; prox; freqs
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]:
flushedFiles=[_5r5_Lucene41_0.pos, _5r5.fdx, _5r5.fnm, _5r5.fdt,
_5r5_Lucene41_0.tim, _5r5_Lucene41_0.tip, _5r5_Lucene41_0.doc]
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed
codec=Lucene410
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flushed:
segment=_5r5 ramUsed=75.564 MB newFlushedSize(includes docstores)=19.546 MB
docs/MB=1,239.201
14:16:50,387 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DWPT][recoveryExecutor-7-thread-1]: flush:
write 1 deletes gen=-1
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
publishFlushedSegment seg-private updates=null
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
publishFlushedSegment
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: push
deletes  24222 deleted terms (unique count=24221) bytesUsed=286752 delGen=8
packetCount=4 totBytesUsed=1259648
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: publish
sets newSegment delGen=9 seg=_5r5(4.10.0):C24222/1:delGen=1
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1370002:delGen=56
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93841:delGen=10 _5m7(4.10.0):C122852/31675:delGen=11
_5hm(4.10.0):C457977/32535:delGen=11 _5q0(4.10.0):C13610/649:delGen=6
_5kb(4.10.0):C424868/19149:delGen=11 _5f5(4.10.0):C116528/42495:delGen=1
_5nx(4.10.0):C33236/20668:delGen=1 _5qm(4.10.0):C29770/1:delGen=1
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/648:delGen=1 _5qv(4.10.0):C3973
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828 _5qh(4.10.0):C1765 _5qi(4.10.0):C1241 _5qq(4.10.0):C1997
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729 _5qo(4.10.0):C3456/1:delGen=1
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802 _5r2(4.10.0):C32769/1:delGen=1 _5r3(4.10.0):C26057
_5r4(4.10.0):C23934/1:delGen=1 _5r5(4.10.0):C24222/1:delGen=1" [34 segments
; isCommit = false]
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
deletes during flush
14:16:50,388 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
applyDeletes: infos=[_4qe(4.10.0):C4312879/1370002:delGen=56,
_554(4.10.0):C3995865/780418:delGen=23,
_56u(4.10.0):C286775/11906:delGen=15, _5co(4.10.0):C871785/93841:delGen=10,
_5m7(4.10.0):C122852/31675:delGen=11, _5hm(4.10.0):C457977/32535:delGen=11,
_5q0(4.10.0):C13610/649:delGen=6, _5kb(4.10.0):C424868/19149:delGen=11,
_5f5(4.10.0):C116528/42495:delGen=1, _5nx(4.10.0):C33236/20668:delGen=1,
_5qm(4.10.0):C29770/1:delGen=1, _5o8(4.10.0):C27155/7531:delGen=1,
_5of(4.10.0):C38545/5677:delGen=1, _5p7(4.10.0):C37457/648:delGen=1,
_5qv(4.10.0):C3973, _5q1(4.10.0):C402/1:delGen=1, _5q2(4.10.0):C779,
_5qa(4.10.0):C967, _5qc(4.10.0):C1828, _5qh(4.10.0):C1765,
_5qi(4.10.0):C1241, _5qq(4.10.0):C1997, _5qr(4.10.0):C1468,
_5qp(4.10.0):C1729, _5qo(4.10.0):C3456/1:delGen=1, _5qu(4.10.0):C27,
_5qt(4.10.0):C30, _5qx(4.10.0):C638, _5qy(4.10.0):C1407, _5qw(4.10.0):C802,
_5r2(4.10.0):C32769/1:delGen=1, _5r3(4.10.0):C26057,
_5r4(4.10.0):C23934/1:delGen=1, _5r5(4.10.0):C24222/1:delGen=1]
packetCount=4
14:16:50,402 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
seg=_5r4(4.10.0):C23934/1:delGen=1 segGen=7
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"_5r2_1.del"
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1nts ; isCommit = false]
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"_5r4_1.del"
14:16:52,774 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 0 msec to
checkpoint
14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
recoveryExecutor-7-thread-1 finishFullFlush success=true
14:16:52,775 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
findMerges: 34 segments
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
large]
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
large]
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
14:16:52,777 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r3(4.10.0):C26057 size=23.893 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qq(4.10.0):C1997 size=1.781 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q2(4.10.0):C779 size=1.554 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
14:16:52,778 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qr(4.10.0):C1468 size=1.390 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qp(4.10.0):C1729/1:delGen=1 size=1.351 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qa(4.10.0):C967 size=1.235 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qi(4.10.0):C1241 size=1.146 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qy(4.10.0):C1407 size=1.050 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5q1(4.10.0):C402/1:delGen=1 size=0.954 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qw(4.10.0):C802/1:delGen=1 size=0.821 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qx(4.10.0):C638 size=0.818 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qt(4.10.0):C30 size=0.072 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
seg=_5qu(4.10.0):C27 size=0.063 MB [floored]
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
allowedSegmentCount=31 vs count=34 (eligible count=32) tooBigCount=2
14:16:52,779 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5co(4.10.0):C871785/93995:delGen=11
_5kb(4.10.0):C424868/49572:delGen=12 _5hm(4.10.0):C457977/83353:delGen=12
_56u(4.10.0):C286775/11906:delGen=15 _5f5(4.10.0):C116528/43621:delGen=2
_5m7(4.10.0):C122852/54010:delGen=12 _5p7(4.10.0):C37457/649:delGen=2
_5of(4.10.0):C38545/5677:delGen=1 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 score=0.7313777596572674 skew=0.341
nonDelRatio=0.852 tooLarge=false size=2506.203 MB
.....

14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
maybe=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 score=0.22289855634027114 skew=0.100 nonDelRatio=1.000
tooLarge=false size=8.907 MB
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:   add
merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 size=8.907 MB score=0.223 skew=0.100 nonDelRatio=1.000
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [TMP][recoveryExecutor-7-thread-1]:
allowedSegmentCount=31 vs count=34 (eligible count=22) tooBigCount=2
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: add merge
to pendingMerges: _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
_5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
_5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
_5qt(4.10.0):C30 _5qu(4.10.0):C27 [total 1 pending]
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge merging= []
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qr(4.10.0):C1468
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qp(4.10.0):C1729/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qa(4.10.0):C967
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qi(4.10.0):C1241
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qy(4.10.0):C1407
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5q1(4.10.0):C402/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qw(4.10.0):C802/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qx(4.10.0):C638
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qt(4.10.0):C30
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
registerMerge info=_5qu(4.10.0):C27
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: now merge
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   index:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
consider merge _5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1
_5qa(4.10.0):C967 _5qi(4.10.0):C1241 _5qy(4.10.0):C1407
_5q1(4.10.0):C402/1:delGen=1 _5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638
_5qt(4.10.0):C30 _5qu(4.10.0):C27
14:16:52,783 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:
launch new thread [Lucene Merge Thread #0]
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]: set
priority of merge thread Lucene Merge Thread #0 to 6
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [CMS][recoveryExecutor-7-thread-1]:   no more
merges pending; now return
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
startCommit(): start
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
startCommit index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 changeCount=28
14:16:52,784 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: start
14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [BD][Lucene Merge Thread #0]: applyDeletes: no deletes; skipping
14:16:52,785 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [BD][Lucene Merge Thread #0]: prune sis=segments_9t:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
14:16:52,788 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge seg=_5r6 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:52,789 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: now merge
  merge=_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
  index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merging _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qr(4.10.0):C1468 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qp(4.10.0):C1729/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qa(4.10.0):C967 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qi(4.10.0):C1241 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qy(4.10.0):C1407 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5q1(4.10.0):C402/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qw(4.10.0):C802/1:delGen=1
delCount=1
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qx(4.10.0):C638 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qt(4.10.0):C30 no deletes
14:16:52,790 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: seg=_5qu(4.10.0):C27 no deletes
14:16:52,792 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: merge store matchedCount=10 vs 10
14:16:52,929 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 137 msec to merge stored fields
[8708 docs]
14:16:53,012 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: done all
syncs: [_5p7_Lucene41_0.pos, _5qr.si, _5kb_Lucene41_0.tim, _5q1.fdt,
_5kb_Lucene41_0.tip, _5qp.fnm, _5o8_1.del, _5qa_Lucene41_0.doc, _5qq.fnm,
_5qr.fnm, _5co_b.del, _5qh.fdx, _5qm.fdx, _5qh.fdt, _5kb_c.del, _5r5.fnm,
_554.fdx, _5qm.fdt, _5r3_Lucene41_0.doc, _4qe.fdt, _56u_Lucene41_0.doc,
_554.fdt, _5qi.fdt, _5q2_Lucene41_0.doc, _4qe.fdx, _5qa.si, _5qi.fdx, _
5qy.si, _56u.fdx, _5qv_Lucene41_0.tip, _5qh.si, _5qw_Lucene41_0.pos,
_5hm_Lucene41_0.pos, _5nx.si, _5co_Lucene41_0.pos, _5q1.fdx, _5qt.fdx,
_5qt.fdt, _5r5.si, _5co.fdt, _5nx.fnm, _5qv_Lucene41_0.tim, _5co.fdx,
_5qh_Lucene41_0.doc, _5kb.fdx, _5qh_1.del, _5qm_Lucene41_0.tim, _56u.fdt,
_5r4_2.del, _5r3_Lucene41_0.tip, _5hm.fdt, _5qv_1.del, _5qm_Lucene41_0.tip,
_5qr_Lucene41_0.doc, _5kb.fdt, _5of_Lucene41_0.tip, _5hm.fdx,
_5r3_Lucene41_0.tim, _5of_Lucene41_0.tim, _56u_f.del, _5qy_Lucene41_0.doc,
_5r4_Lucene41_0.pos, _5qq.si, _5qt_Lucene41_0.doc, _5qq_Lucene41_0.pos,
_5qa.fnm, _5qv.si, _5r2.fnm, _5qo_2.del, _5p7.fdt, _5r3.fnm,
_5p7_Lucene41_0.doc, _5r4.fdt, _5qo.si, _5p7.fdx, _5qa_Lucene41_0.tim, _
5qp.si, _5kb.si, _5r4.fdx, _5qp_Lucene41_0.tip, _4qe.si,
_5qp_Lucene41_0.tim, _5qa_Lucene41_0.tip, _4qe_1l.del, _5qp_1.del,
_5qw_1.del, _5qm_2.del, _5m7_Lucene41_0.tim, _5qw.si, _5m7_Lucene41_0.tip, _
56u.si, _5of_Lucene41_0.doc, _5of.si, _5of.fnm, _4qe_Lucene41_0.tip,
_5qh_Lucene41_0.pos, _5qi.si, _4qe_Lucene41_0.tim, _5qc.fdx,
_5r2_Lucene41_0.pos, _5qm.fnm, _5qc.fdt, _5qx_Lucene41_0.pos,
_5qo_Lucene41_0.doc, _5q0.fnm, _5qv.fdx, _5qi_Lucene41_0.pos,
_5p7_Lucene41_0.tim, _5qv.fdt, _5f5.si, _5p7_Lucene41_0.tip, _5q2.fdt, _
5o8.si, _5qm_Lucene41_0.doc, _5qc_Lucene41_0.tim, _5f5_Lucene41_0.tim,
_5f5_Lucene41_0.tip, _5qc_Lucene41_0.tip, _5qx.fdx, _554_Lucene41_0.doc,
_5q2_Lucene41_0.pos, _5r4_Lucene41_0.tim, _5qx.fdt, _5q1_Lucene41_0.pos,
_5qp_Lucene41_0.doc, _5qm.si, _56u.fnm, _5q0_Lucene41_0.pos,
_5r4_Lucene41_0.tip, _5qq.fdx, _5qr_Lucene41_0.pos, _56u_Lucene41_0.pos,
_5r2_Lucene41_0.doc, _5qw_Lucene41_0.doc, _5r5_Lucene41_0.tim,
_5r5_Lucene41_0.tip, _554.fnm, _5qi.fnm, _56u_Lucene41_0.tim,
_5qo_Lucene41_0.pos, _56u_Lucene41_0.tip, _5f5.fnm, _5qq.fdt, _5q0_7.del,
_5m7.fnm, _5qo.fnm, _5qu.si, _5q2.fdx, _5qt_Lucene41_0.tip,
_5qw_Lucene41_0.tip, _5qt_Lucene41_0.tim, _5qc.si, _5hm.si, _5m7.si, _5r2.si,
_5f5_Lucene41_0.pos, _5qy_Lucene41_0.tip, _5nx_Lucene41_0.tim,
_5r2_Lucene41_0.tip, _5qm_Lucene41_0.pos, _5qq_Lucene41_0.tim, _5co.si,
_5r2.fdt, _5r2_Lucene41_0.tim, _5f5_2.del, _5qq_Lucene41_0.tip,
_5qy_Lucene41_0.tim, _5nx_Lucene41_0.tip, _5o8_Lucene41_0.doc, _5r3.fdt,
_5qv_Lucene41_0.pos, _5q1_Lucene41_0.doc, _5qo_Lucene41_0.tip, _5qx.fnm,
_5qo_Lucene41_0.tim, _5co.fnm, _5qu.fdt, _5qu.fdx, _5r5.fdx, _554_n.del,
_4qe_Lucene41_0.pos, _5q1_Lucene41_0.tip, _5f5_Lucene41_0.doc, _5r5.fdt,
_5qw_Lucene41_0.tim, _5m7_c.del, _5qv.fnm, _5q1_Lucene41_0.tim, _5kb.fnm, _
5q2.si, _5qy.fnm, _5kb_Lucene41_0.pos, _5qx_Lucene41_0.doc, _5r4.fnm,
_5qu_Lucene41_0.pos, _5r5_Lucene41_0.doc, _554_Lucene41_0.pos,
_5qc_Lucene41_0.doc, _5hm_Lucene41_0.doc, _5p7.fnm, _5qt.si, _5p7_2.del,
_5qi_Lucene41_0.doc, _5r2_2.del, _5r3.fdx, _5o8.fdx, _5q1.fnm, _5nx.fdt,
_4qe.fnm, _5nx.fdx, _5q1.si, _5r3_Lucene41_0.pos, _5qa.fdt, _5r2.fdx,
_5o8.fdt, _5qa_Lucene41_0.pos, _5qc_1.del, _5qw.fnm, _5m7_Lucene41_0.doc,
_5qa.fdx, _5hm.fnm, _554.si, _5r4.si, _5qy_Lucene41_0.pos,
_5qr_Lucene41_0.tim, _5qr_Lucene41_0.tip, _5qy.fdx, _5nx_Lucene41_0.pos,
_5m7.fdt, _5kb_Lucene41_0.doc, _5m7.fdx, _554_Lucene41_0.tip, _5qc.fnm,
_5o8_Lucene41_0.tim, _5co_Lucene41_0.doc, _554_Lucene41_0.tim, _5qy.fdt, _
5qx.si, _5o8_Lucene41_0.tip, _5qu.fnm, _5nx_Lucene41_0.doc, _5q2.fnm,
_5o8_Lucene41_0.pos, _5qt_Lucene41_0.pos, _5qt.fnm, _5qh_Lucene41_0.tip,
_5qh_Lucene41_0.tim, _5q2_Lucene41_0.tip, _5q2_Lucene41_0.tim,
_5q0_Lucene41_0.doc, _5hm_c.del, _5r5_1.del, _5o8.fnm, _5qu_Lucene41_0.doc,
_5q0_Lucene41_0.tip, _5hm_Lucene41_0.tip, _5r5_Lucene41_0.pos, _5qo.fdx,
_5f5.fdt, _5q0_Lucene41_0.tim, _5qr.fdx, _5qu_Lucene41_0.tim, _5p7.si,
_5q1_1.del, _5of_1.del, _5qp.fdt, _5q0.fdx, _5q0.si, _5r4_Lucene41_0.doc,
_5of.fdx, _4qe_Lucene41_0.doc, _5qh.fnm, _5of.fdt, _5of_Lucene41_0.pos,
_5f5.fdx, _5m7_Lucene41_0.pos, _5r3.si, _5qw.fdt, _5qx_Lucene41_0.tim,
_5qx_Lucene41_0.tip, _5nx_2.del, _5qv_Lucene41_0.doc, _5qw.fdx, _5q0.fdt,
_5co_Lucene41_0.tim, _5qq_Lucene41_0.doc, _5qp.fdx, _5qu_Lucene41_0.tip,
_5qr.fdt, _5qi_Lucene41_0.tip, _5qp_Lucene41_0.pos, _5hm_Lucene41_0.tim,
_5qi_Lucene41_0.tim, _5qo.fdt, _5co_Lucene41_0.tip, _5qc_Lucene41_0.pos]
14:16:53,013 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
pendingCommit != null
14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
wrote segments file "segments_9u"
14:16:53,015 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: now
checkpoint "_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1" [34 segments ; isCommit = true]
14:16:53,015 INFO  [org.apache.solr.core.SolrCore]
(recoveryExecutor-7-thread-1) SolrDeletionPolicy.onCommit: commits: num=2
        commit{dir=NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9t,generation=353}
        commit{dir=NRTCachingDirectory(MMapDirectory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697
lockFactory=NativeFSLockFactory@/opt/solr/solrnodes/solrnode1/search1_shard7_replica4/data/index.20150126140904697;
maxCacheMB=48.0 maxMergeSizeMB=4.0),segFN=segments_9u,generation=354}
14:16:53,016 INFO  [org.apache.solr.core.SolrCore]
(recoveryExecutor-7-thread-1) newest commit generation = 354
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]:
deleteCommits: now decRef commit "segments_9t"
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: delete
"segments_9t"
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IFD][recoveryExecutor-7-thread-1]: 1 msec to
checkpoint
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
took 3733.0 msec
14:16:53,016 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: commit:
done
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=0 deletes=false hasTickets:false
pendingChangesInFullFlush: false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]:
nrtIsCurrent: infoVersion matches: false; DW changes: false; BD changes:
false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: flush at
getReader
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
startFullFlush
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
anyChanges? numDocsInRam=0 deletes=false hasTickets:false
pendingChangesInFullFlush: false
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: apply all
deletes during flush
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]:
applyDeletes: no deletes; skipping
14:16:53,017 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [BD][recoveryExecutor-7-thread-1]: prune
sis=segments_9u: _4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1 minGen=10 packetCount=0
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: return
reader version=16082 reader=StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [DW][recoveryExecutor-7-thread-1]:
recoveryExecutor-7-thread-1 finishFullFlush success=true
14:16:53,026 INFO  [org.apache.solr.update.LoggingInfoStream]
(recoveryExecutor-7-thread-1) [IW][recoveryExecutor-7-thread-1]: getReader
took 9 msec
14:16:53,026 INFO  [org.apache.solr.search.SolrIndexSearcher]
(recoveryExecutor-7-thread-1) Opening Searcher@dbe17f9[search1_shard7_replica4]
main
14:16:53,120 INFO  [org.apache.solr.update.UpdateHandler]
(recoveryExecutor-7-thread-1) end_commit_flush

==> gc.20150126-135638.log <==
1214.654: [GC1214.654: [ParNew: 974509K->56784K(996800K), 0.0150450 secs]
1519683K->603424K(1995752K), 0.0152330 secs] [Times: user=0.14 sys=0.00,
real=0.02 secs]

==> server.log <==
14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 477 msec to merge postings [8708
docs]
14:16:53,406 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [SM][Lucene Merge Thread #0]: 0 msec to merge doc values [8708
docs]
14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge codec=Lucene410
docCount=8708; merged segment has no vectors; no norms; no docValues; prox;
freqs
14:16:53,407 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merged segment size=7.623 MB vs
estimate=8.906 MB
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: commitMerge: _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967 _5qi(4.10.0):C1241
_5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27 index=_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: commitMergeDeletes
_5qr(4.10.0):C1468 _5qp(4.10.0):C1729/1:delGen=1 _5qa(4.10.0):C967
_5qi(4.10.0):C1241 _5qy(4.10.0):C1407 _5q1(4.10.0):C402/1:delGen=1
_5qw(4.10.0):C802/1:delGen=1 _5qx(4.10.0):C638 _5qt(4.10.0):C30
_5qu(4.10.0):C27
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: no new deletes or field updates
since merge started
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IFD][Lucene Merge Thread #0]: now checkpoint
"_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
_5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
_5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1" [25 segments ; isCommit = false]
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IFD][Lucene Merge Thread #0]: 0 msec to checkpoint
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: after commitMerge:
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5r6(4.10.0):C8708 _5q2(4.10.0):C779 _5qc(4.10.0):C1828/1:delGen=1
_5qh(4.10.0):C1765/1:delGen=1 _5qq(4.10.0):C1997
_5qo(4.10.0):C3456/2:delGen=2 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1
14:16:53,408 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]: findMerges: 25 segments
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_554(4.10.0):C3995865/780418:delGen=23 size=3669.307 MB [skip: too
large]
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_4qe(4.10.0):C4312879/1370113:delGen=57 size=3506.254 MB [skip: too
large]
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5co(4.10.0):C871785/93995:delGen=11 size=853.668 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5kb(4.10.0):C424868/49572:delGen=12 size=518.704 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5hm(4.10.0):C457977/83353:delGen=12 size=470.422 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_56u(4.10.0):C286775/11906:delGen=15 size=312.952 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5f5(4.10.0):C116528/43621:delGen=2 size=95.529 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5m7(4.10.0):C122852/54010:delGen=12 size=84.949 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5p7(4.10.0):C37457/649:delGen=2 size=54.241 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5of(4.10.0):C38545/5677:delGen=1 size=50.672 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qm(4.10.0):C29770/4:delGen=2 size=34.052 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5o8(4.10.0):C27155/7531:delGen=1 size=31.008 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r2(4.10.0):C32769/4:delGen=2 size=27.410 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r3(4.10.0):C26057
size=23.893 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r4(4.10.0):C23934/3:delGen=2 size=22.004 MB
14:16:53,409 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5nx(4.10.0):C33236/20669:delGen=2 size=19.861 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5r5(4.10.0):C24222/1:delGen=1 size=19.546 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5q0(4.10.0):C13610/2625:delGen=7 size=12.480 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5r6(4.10.0):C8708
size=7.622 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qv(4.10.0):C3973/1:delGen=1 size=3.402 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qo(4.10.0):C3456/2:delGen=2 size=3.147 MB
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5qq(4.10.0):C1997
size=1.781 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   seg=_5q2(4.10.0):C779
size=1.554 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qh(4.10.0):C1765/1:delGen=1 size=1.549 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:
seg=_5qc(4.10.0):C1828/1:delGen=1 size=1.401 MB [floored]
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [TMP][Lucene Merge Thread #0]:   allowedSegmentCount=31 vs
count=25 (eligible count=23) tooBigCount=2
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [IW][Lucene Merge Thread #0]: merge time 625 msec for 8708 docs
14:16:53,410 INFO  [org.apache.solr.update.LoggingInfoStream] (Lucene Merge
Thread #0) [CMS][Lucene Merge Thread #0]:   merge thread: done

==> gc.20150126-135638.log <==
1217.747: [GC1217.747: [ParNew: 942864K->33781K(996800K), 0.0409520 secs]
1489504K->580420K(1995752K), 0.0411310 secs] [Times: user=0.25 sys=0.00,
real=0.04 secs]

==> server.log <==
14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) QuerySenderListener sending requests to
Searcher@dbe17f9[search1_shard7_replica4]
main{StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)}
14:16:58,363 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) QuerySenderListener done.
14:16:58,365 INFO  [org.apache.solr.core.SolrCore]
(searcherExecutor-6-thread-1) [search1_shard7_replica4] Registered new
searcher Searcher@dbe17f9[search1_shard7_replica4]
main{StandardDirectoryReader(segments_9u:16082:nrt
_4qe(4.10.0):C4312879/1370113:delGen=57
_554(4.10.0):C3995865/780418:delGen=23 _56u(4.10.0):C286775/11906:delGen=15
_5co(4.10.0):C871785/93995:delGen=11 _5m7(4.10.0):C122852/54010:delGen=12
_5hm(4.10.0):C457977/83353:delGen=12 _5q0(4.10.0):C13610/2625:delGen=7
_5kb(4.10.0):C424868/49572:delGen=12 _5f5(4.10.0):C116528/43621:delGen=2
_5nx(4.10.0):C33236/20669:delGen=2 _5qm(4.10.0):C29770/4:delGen=2
_5o8(4.10.0):C27155/7531:delGen=1 _5of(4.10.0):C38545/5677:delGen=1
_5p7(4.10.0):C37457/649:delGen=2 _5qv(4.10.0):C3973/1:delGen=1
_5q1(4.10.0):C402/1:delGen=1 _5q2(4.10.0):C779 _5qa(4.10.0):C967
_5qc(4.10.0):C1828/1:delGen=1 _5qh(4.10.0):C1765/1:delGen=1
_5qi(4.10.0):C1241 _5qq(4.10.0):C1997 _5qr(4.10.0):C1468
_5qp(4.10.0):C1729/1:delGen=1 _5qo(4.10.0):C3456/2:delGen=2
_5qu(4.10.0):C27 _5qt(4.10.0):C30 _5qx(4.10.0):C638 _5qy(4.10.0):C1407
_5qw(4.10.0):C802/1:delGen=1 _5r2(4.10.0):C32769/4:delGen=2
_5r3(4.10.0):C26057 _5r4(4.10.0):C23934/3:delGen=2
_5r5(4.10.0):C24222/1:delGen=1)}
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_4qe_1k.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5p7_1.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5nx_1.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5co_a.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [IFD][searcherExecutor-6-thread-1]: delete
"_5q0_6.del"
14:16:58,367 INFO  [org.apache.solr.update.LoggingInfoStream]
(searcherExecutor-6-thread-1) [

On Sun, Jan 25, 2015 at 9:53 PM, Erick Erickson <er...@gmail.com>
wrote:

> Ah, OK. Whew! because I was wondering how you were running at _all_ if all
> the memory was allocated to the JVM ;)..
>
> What is your Zookeeper timeout? The original default was 15 seconds and
> this
> has caused problems like this. Here's the scenario:
> You send a bunch of docs at the server, and eventually you hit a
> stop-the-world
> GC that takes longer than the Zookeeper timeout. So ZK thinks the node is
> down
> and initiates recovery. Eventually, you hit this on all the replicas.
>
> Sometimes I've seen situations where the answer is giving a bit more memory
> to the JVM, say 2-4G in your case. The theory here (and this is a shot in
> the
> dark) that your peak JVM requirements are close to your 12G, so the garbage
> collection spends enormous amounts of time collecting a small bit of
> memory,
> runs for some fraction of a second and does it again. Adding more to the
> JVMs
> memory allows the parallel collections to work without so many
> stop-the-world
> GC pauses.
>
> So what I'd do is turn on GC logging (probably on the replicas) and look
> for
> very long GC pauses. Mark Miller put together a blog here:
> https://lucidworks.com/blog/garbage-collection-bootcamp-1-0/
>
> See the "getting a view into garbage collection". The smoking gun here
> is if you see full GC pauses that are longer than the ZK timeout.
>
> 90M docs in 4 hours across 10 shards is only 625/sec or so per shard. I've
> seen
> sustained indexing rates significantly above this, YMMV or course, a lot
> depends
> on the size of the docs.
>
> What version of Solr BTW? And when you say you fire a bunch of indexers,
> I'm
> assuming these are SolrJ clients and use CloudSolrServer?
>
> Best,
> Erick
>
>
> On Sun, Jan 25, 2015 at 4:10 PM, Vijay Sekhri <se...@gmail.com>
> wrote:
>
> > Thank you for the reply Eric.
> > I am sorry I had wrong information posted. I posted our DEV env
> > configuration by mistake.
> > After double checking our stress and Prod Beta env where we have found
> the
> > original issue, I found all the searchers have around 50 GB of RAM
> > available and two instances of JVM running (2 different ports). Both
> > instances have 12 GB allocated. The rest 26 GB is available for the OS.
> 1st
> >  instance on a host has search1 collection (live collection) and the 2nd
> > instance on the same host  has search2 collection (for full indexing ).
> >
> > There is plenty room for OS related tasks. Our issue is not in anyway
> > related to OS starving as shown from our dashboards.
> > We have been through
> >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > a lot of times but  we have two modes of operation
> > a)  1st collection (Live traffic) - heavy searches and medium indexing
> > b)  2nd collection (Not serving traffic) - very heavy indexing, no
> searches
> >
> > When our indexing finishes we swap the alias for these collection . So
> > essentially we need to have a configuration that can support both the use
> > cases together. We have tried a lot of different configuration options
> and
> > none of them seems to work. My suspicion is that solr cloud is unable to
> > keep up with the updates at the rate we are sending while it is trying to
> > be consistent with all the replicas.
> >
> >
> > On Sun, Jan 25, 2015 at 5:30 PM, Erick Erickson <erickerickson@gmail.com
> >
> > wrote:
> >
> > > Shawn directed you over here to the user list, but I see this note on
> > > SOLR-7030:
> > > "All our searchers have 12 GB of RAM available and have quad core
> > Intel(R)
> > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> > > jboss and solr in it . All 12 GB is available as heap for the java
> > > process..."
> > >
> > > So you have 12G physical memory and have allocated 12G to the Java
> > process?
> > > This is an anti-pattern. If that's
> > > the case, your operating system is being starved for memory, probably
> > > hitting a state where it spends all of its
> > > time in stop-the-world garbage collection, eventually it doesn't
> respond
> > to
> > > Zookeeper's ping so Zookeeper
> > > thinks the node is down and puts it into recovery. Where it spends a
> lot
> > of
> > > time doing... essentially nothing.
> > >
> > > About the hard and soft commits: I suspect these are entirely
> unrelated,
> > > but here's a blog on what they do, you
> > > should pick the configuration that supports your use case (i.e. how
> much
> > > latency can you stand between indexing
> > > and being able to search?).
> > >
> > >
> > >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> > >
> > > Here's one very good reason you shouldn't starve your op system by
> > > allocating all the physical memory to the JVM:
> > >
> http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html
> > >
> > >
> > > But your biggest problem is that you have far too much of your physical
> > > memory allocated to the JVM. This
> > > will cause you endless problems, you just need more physical memory on
> > > those boxes. It's _possible_ you could
> > > get by with less memory for the JVM, counterintuitive as it seems try
> 8G
> > or
> > > maybe even 6G. At some point
> > > you'll hit OOM errors, but that'll give you a lower limit on what the
> JVM
> > > needs.
> > >
> > > Unless I've mis-interpreted what you've written, though, I doubt you'll
> > get
> > > stable with that much memory allocated
> > > to the JVM.
> > >
> > > Best,
> > > Erick
> > >
> > >
> > >
> > > On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <se...@gmail.com>
> > > wrote:
> > >
> > > > We have a cluster of solr cloud server with 10 shards and 4 replicas
> in
> > > > each shard in our stress environment. In our prod environment we will
> > > have
> > > > 10 shards and 15 replicas in each shard. Our current commit settings
> > are
> > > as
> > > > follows
> > > >
> > > > *    <autoSoftCommit>*
> > > > *        <maxDocs>500000</maxDocs>*
> > > > *        <maxTime>180000</maxTime>*
> > > > *    </autoSoftCommit>*
> > > > *    <autoCommit>*
> > > > *        <maxDocs>2000000</maxDocs>*
> > > > *        <maxTime>180000</maxTime>*
> > > > *        <openSearcher>false</openSearcher>*
> > > > *    </autoCommit>*
> > > >
> > > >
> > > > We indexed roughly 90 Million docs. We have two different ways to
> index
> > > > documents a) Full indexing. It takes 4 hours to index 90 Million docs
> > and
> > > > the rate of docs coming to the searcher is around 6000 per second b)
> > > > Incremental indexing. It takes an hour to indexed delta changes.
> > Roughly
> > > > there are 3 million changes and rate of docs coming to the searchers
> is
> > > > 2500
> > > > per second
> > > >
> > > > We have two collections search1 and search2. When we do full
> indexing ,
> > > we
> > > > do it in search2 collection while search1 is serving live traffic.
> > After
> > > it
> > > > finishes we swap the collection using aliases so that the search2
> > > > collection serves live traffic while search1 becomes available for
> next
> > > > full indexing run. When we do incremental indexing we do it in the
> > > search1
> > > > collection which is serving live traffic.
> > > >
> > > > All our searchers have 12 GB of RAM available and have quad core
> > Intel(R)
> > > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running
> i.e
> > > > jboss and solr in it . All 12 GB is available as heap for the java
> > > > process.  We have observed that the heap memory of the java process
> > > average
> > > > around 8 - 10 GB. All searchers have final index size of 9 GB. So in
> > > total
> > > > there are 9X10 (shards) =  90GB worth of index files.
> > > >
> > > >  We have observed the following issue when we trigger indexing . In
> > about
> > > > 10 minutes after we trigger indexing on 14 parallel hosts, the
> replicas
> > > > goes in to recovery mode. This happens to all the shards . In about
> 20
> > > > minutes more and more replicas start going into recovery mode. After
> > > about
> > > > half an hour all replicas except the leader are in recovery mode. We
> > > cannot
> > > > throttle the indexing load as that will increase our overall indexing
> > > time.
> > > > So to overcome this issue, we remove all the replicas before we
> trigger
> > > the
> > > > indexing and then add them back after the indexing finishes.
> > > >
> > > > We observe the same behavior of replicas going into recovery when we
> do
> > > > incremental indexing. We cannot remove replicas during our
> incremental
> > > > indexing because it is also serving live traffic. We tried to
> throttle
> > > our
> > > > indexing speed , however the cluster still goes into recovery .
> > > >
> > > > If we leave the cluster as it , when the indexing finishes , it
> > > eventually
> > > > recovers after a while. As it is serving live traffic we cannot have
> > > these
> > > > replicas go into recovery mode because it degrades the search
> > performance
> > > > also , our tests have shown.
> > > >
> > > > We have tried different commit settings like below
> > > >
> > > > a) No auto soft commit, no auto hard commit and a commit triggered at
> > the
> > > > end of indexing b) No auto soft commit, yes auto hard commit and a
> > commit
> > > > in the end of indexing
> > > > c) Yes auto soft commit , no auto hard commit
> > > > d) Yes auto soft commit , yes auto hard commit
> > > > e) Different frequency setting for commits for above. Please NOTE
> that
> > we
> > > > have tried 15 minute soft commit setting and 30 minutes hard commit
> > > > settings. Same time settings for both, 30 minute soft commit and an
> > hour
> > > > hard commit setting
> > > >
> > > > Unfortunately all the above yields the same behavior . The replicas
> > still
> > > > goes in recovery We have increased the zookeeper timeout from 30
> > seconds
> > > to
> > > > 5 minutes and the problem persists. Is there any setting that would
> fix
> > > > this issue ?
> > > >
> > > > --
> > > > *********************************************
> > > > Vijay Sekhri
> > > > *********************************************
> > > >
> > >
> >
> >
> >
> > --
> > *********************************************
> > Vijay Sekhri
> > *********************************************
> >
>



-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Erick Erickson <er...@gmail.com>.
Ah, OK. Whew! because I was wondering how you were running at _all_ if all
the memory was allocated to the JVM ;)..

What is your Zookeeper timeout? The original default was 15 seconds and this
has caused problems like this. Here's the scenario:
You send a bunch of docs at the server, and eventually you hit a
stop-the-world
GC that takes longer than the Zookeeper timeout. So ZK thinks the node is
down
and initiates recovery. Eventually, you hit this on all the replicas.

Sometimes I've seen situations where the answer is giving a bit more memory
to the JVM, say 2-4G in your case. The theory here (and this is a shot in
the
dark) that your peak JVM requirements are close to your 12G, so the garbage
collection spends enormous amounts of time collecting a small bit of memory,
runs for some fraction of a second and does it again. Adding more to the
JVMs
memory allows the parallel collections to work without so many
stop-the-world
GC pauses.

So what I'd do is turn on GC logging (probably on the replicas) and look for
very long GC pauses. Mark Miller put together a blog here:
https://lucidworks.com/blog/garbage-collection-bootcamp-1-0/

See the "getting a view into garbage collection". The smoking gun here
is if you see full GC pauses that are longer than the ZK timeout.

90M docs in 4 hours across 10 shards is only 625/sec or so per shard. I've
seen
sustained indexing rates significantly above this, YMMV or course, a lot
depends
on the size of the docs.

What version of Solr BTW? And when you say you fire a bunch of indexers,
I'm
assuming these are SolrJ clients and use CloudSolrServer?

Best,
Erick


On Sun, Jan 25, 2015 at 4:10 PM, Vijay Sekhri <se...@gmail.com> wrote:

> Thank you for the reply Eric.
> I am sorry I had wrong information posted. I posted our DEV env
> configuration by mistake.
> After double checking our stress and Prod Beta env where we have found the
> original issue, I found all the searchers have around 50 GB of RAM
> available and two instances of JVM running (2 different ports). Both
> instances have 12 GB allocated. The rest 26 GB is available for the OS. 1st
>  instance on a host has search1 collection (live collection) and the 2nd
> instance on the same host  has search2 collection (for full indexing ).
>
> There is plenty room for OS related tasks. Our issue is not in anyway
> related to OS starving as shown from our dashboards.
> We have been through
>
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> a lot of times but  we have two modes of operation
> a)  1st collection (Live traffic) - heavy searches and medium indexing
> b)  2nd collection (Not serving traffic) - very heavy indexing, no searches
>
> When our indexing finishes we swap the alias for these collection . So
> essentially we need to have a configuration that can support both the use
> cases together. We have tried a lot of different configuration options and
> none of them seems to work. My suspicion is that solr cloud is unable to
> keep up with the updates at the rate we are sending while it is trying to
> be consistent with all the replicas.
>
>
> On Sun, Jan 25, 2015 at 5:30 PM, Erick Erickson <er...@gmail.com>
> wrote:
>
> > Shawn directed you over here to the user list, but I see this note on
> > SOLR-7030:
> > "All our searchers have 12 GB of RAM available and have quad core
> Intel(R)
> > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> > jboss and solr in it . All 12 GB is available as heap for the java
> > process..."
> >
> > So you have 12G physical memory and have allocated 12G to the Java
> process?
> > This is an anti-pattern. If that's
> > the case, your operating system is being starved for memory, probably
> > hitting a state where it spends all of its
> > time in stop-the-world garbage collection, eventually it doesn't respond
> to
> > Zookeeper's ping so Zookeeper
> > thinks the node is down and puts it into recovery. Where it spends a lot
> of
> > time doing... essentially nothing.
> >
> > About the hard and soft commits: I suspect these are entirely unrelated,
> > but here's a blog on what they do, you
> > should pick the configuration that supports your use case (i.e. how much
> > latency can you stand between indexing
> > and being able to search?).
> >
> >
> >
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
> >
> > Here's one very good reason you shouldn't starve your op system by
> > allocating all the physical memory to the JVM:
> > http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html
> >
> >
> > But your biggest problem is that you have far too much of your physical
> > memory allocated to the JVM. This
> > will cause you endless problems, you just need more physical memory on
> > those boxes. It's _possible_ you could
> > get by with less memory for the JVM, counterintuitive as it seems try 8G
> or
> > maybe even 6G. At some point
> > you'll hit OOM errors, but that'll give you a lower limit on what the JVM
> > needs.
> >
> > Unless I've mis-interpreted what you've written, though, I doubt you'll
> get
> > stable with that much memory allocated
> > to the JVM.
> >
> > Best,
> > Erick
> >
> >
> >
> > On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <se...@gmail.com>
> > wrote:
> >
> > > We have a cluster of solr cloud server with 10 shards and 4 replicas in
> > > each shard in our stress environment. In our prod environment we will
> > have
> > > 10 shards and 15 replicas in each shard. Our current commit settings
> are
> > as
> > > follows
> > >
> > > *    <autoSoftCommit>*
> > > *        <maxDocs>500000</maxDocs>*
> > > *        <maxTime>180000</maxTime>*
> > > *    </autoSoftCommit>*
> > > *    <autoCommit>*
> > > *        <maxDocs>2000000</maxDocs>*
> > > *        <maxTime>180000</maxTime>*
> > > *        <openSearcher>false</openSearcher>*
> > > *    </autoCommit>*
> > >
> > >
> > > We indexed roughly 90 Million docs. We have two different ways to index
> > > documents a) Full indexing. It takes 4 hours to index 90 Million docs
> and
> > > the rate of docs coming to the searcher is around 6000 per second b)
> > > Incremental indexing. It takes an hour to indexed delta changes.
> Roughly
> > > there are 3 million changes and rate of docs coming to the searchers is
> > > 2500
> > > per second
> > >
> > > We have two collections search1 and search2. When we do full indexing ,
> > we
> > > do it in search2 collection while search1 is serving live traffic.
> After
> > it
> > > finishes we swap the collection using aliases so that the search2
> > > collection serves live traffic while search1 becomes available for next
> > > full indexing run. When we do incremental indexing we do it in the
> > search1
> > > collection which is serving live traffic.
> > >
> > > All our searchers have 12 GB of RAM available and have quad core
> Intel(R)
> > > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> > > jboss and solr in it . All 12 GB is available as heap for the java
> > > process.  We have observed that the heap memory of the java process
> > average
> > > around 8 - 10 GB. All searchers have final index size of 9 GB. So in
> > total
> > > there are 9X10 (shards) =  90GB worth of index files.
> > >
> > >  We have observed the following issue when we trigger indexing . In
> about
> > > 10 minutes after we trigger indexing on 14 parallel hosts, the replicas
> > > goes in to recovery mode. This happens to all the shards . In about 20
> > > minutes more and more replicas start going into recovery mode. After
> > about
> > > half an hour all replicas except the leader are in recovery mode. We
> > cannot
> > > throttle the indexing load as that will increase our overall indexing
> > time.
> > > So to overcome this issue, we remove all the replicas before we trigger
> > the
> > > indexing and then add them back after the indexing finishes.
> > >
> > > We observe the same behavior of replicas going into recovery when we do
> > > incremental indexing. We cannot remove replicas during our incremental
> > > indexing because it is also serving live traffic. We tried to throttle
> > our
> > > indexing speed , however the cluster still goes into recovery .
> > >
> > > If we leave the cluster as it , when the indexing finishes , it
> > eventually
> > > recovers after a while. As it is serving live traffic we cannot have
> > these
> > > replicas go into recovery mode because it degrades the search
> performance
> > > also , our tests have shown.
> > >
> > > We have tried different commit settings like below
> > >
> > > a) No auto soft commit, no auto hard commit and a commit triggered at
> the
> > > end of indexing b) No auto soft commit, yes auto hard commit and a
> commit
> > > in the end of indexing
> > > c) Yes auto soft commit , no auto hard commit
> > > d) Yes auto soft commit , yes auto hard commit
> > > e) Different frequency setting for commits for above. Please NOTE that
> we
> > > have tried 15 minute soft commit setting and 30 minutes hard commit
> > > settings. Same time settings for both, 30 minute soft commit and an
> hour
> > > hard commit setting
> > >
> > > Unfortunately all the above yields the same behavior . The replicas
> still
> > > goes in recovery We have increased the zookeeper timeout from 30
> seconds
> > to
> > > 5 minutes and the problem persists. Is there any setting that would fix
> > > this issue ?
> > >
> > > --
> > > *********************************************
> > > Vijay Sekhri
> > > *********************************************
> > >
> >
>
>
>
> --
> *********************************************
> Vijay Sekhri
> *********************************************
>

Re: replicas goes in recovery mode right after update

Posted by Vijay Sekhri <se...@gmail.com>.
Thank you for the reply Eric.
I am sorry I had wrong information posted. I posted our DEV env
configuration by mistake.
After double checking our stress and Prod Beta env where we have found the
original issue, I found all the searchers have around 50 GB of RAM
available and two instances of JVM running (2 different ports). Both
instances have 12 GB allocated. The rest 26 GB is available for the OS. 1st
 instance on a host has search1 collection (live collection) and the 2nd
instance on the same host  has search2 collection (for full indexing ).

There is plenty room for OS related tasks. Our issue is not in anyway
related to OS starving as shown from our dashboards.
We have been through
https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
a lot of times but  we have two modes of operation
a)  1st collection (Live traffic) - heavy searches and medium indexing
b)  2nd collection (Not serving traffic) - very heavy indexing, no searches

When our indexing finishes we swap the alias for these collection . So
essentially we need to have a configuration that can support both the use
cases together. We have tried a lot of different configuration options and
none of them seems to work. My suspicion is that solr cloud is unable to
keep up with the updates at the rate we are sending while it is trying to
be consistent with all the replicas.


On Sun, Jan 25, 2015 at 5:30 PM, Erick Erickson <er...@gmail.com>
wrote:

> Shawn directed you over here to the user list, but I see this note on
> SOLR-7030:
> "All our searchers have 12 GB of RAM available and have quad core Intel(R)
> Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> jboss and solr in it . All 12 GB is available as heap for the java
> process..."
>
> So you have 12G physical memory and have allocated 12G to the Java process?
> This is an anti-pattern. If that's
> the case, your operating system is being starved for memory, probably
> hitting a state where it spends all of its
> time in stop-the-world garbage collection, eventually it doesn't respond to
> Zookeeper's ping so Zookeeper
> thinks the node is down and puts it into recovery. Where it spends a lot of
> time doing... essentially nothing.
>
> About the hard and soft commits: I suspect these are entirely unrelated,
> but here's a blog on what they do, you
> should pick the configuration that supports your use case (i.e. how much
> latency can you stand between indexing
> and being able to search?).
>
>
> https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/
>
> Here's one very good reason you shouldn't starve your op system by
> allocating all the physical memory to the JVM:
> http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html
>
>
> But your biggest problem is that you have far too much of your physical
> memory allocated to the JVM. This
> will cause you endless problems, you just need more physical memory on
> those boxes. It's _possible_ you could
> get by with less memory for the JVM, counterintuitive as it seems try 8G or
> maybe even 6G. At some point
> you'll hit OOM errors, but that'll give you a lower limit on what the JVM
> needs.
>
> Unless I've mis-interpreted what you've written, though, I doubt you'll get
> stable with that much memory allocated
> to the JVM.
>
> Best,
> Erick
>
>
>
> On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <se...@gmail.com>
> wrote:
>
> > We have a cluster of solr cloud server with 10 shards and 4 replicas in
> > each shard in our stress environment. In our prod environment we will
> have
> > 10 shards and 15 replicas in each shard. Our current commit settings are
> as
> > follows
> >
> > *    <autoSoftCommit>*
> > *        <maxDocs>500000</maxDocs>*
> > *        <maxTime>180000</maxTime>*
> > *    </autoSoftCommit>*
> > *    <autoCommit>*
> > *        <maxDocs>2000000</maxDocs>*
> > *        <maxTime>180000</maxTime>*
> > *        <openSearcher>false</openSearcher>*
> > *    </autoCommit>*
> >
> >
> > We indexed roughly 90 Million docs. We have two different ways to index
> > documents a) Full indexing. It takes 4 hours to index 90 Million docs and
> > the rate of docs coming to the searcher is around 6000 per second b)
> > Incremental indexing. It takes an hour to indexed delta changes. Roughly
> > there are 3 million changes and rate of docs coming to the searchers is
> > 2500
> > per second
> >
> > We have two collections search1 and search2. When we do full indexing ,
> we
> > do it in search2 collection while search1 is serving live traffic. After
> it
> > finishes we swap the collection using aliases so that the search2
> > collection serves live traffic while search1 becomes available for next
> > full indexing run. When we do incremental indexing we do it in the
> search1
> > collection which is serving live traffic.
> >
> > All our searchers have 12 GB of RAM available and have quad core Intel(R)
> > Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> > jboss and solr in it . All 12 GB is available as heap for the java
> > process.  We have observed that the heap memory of the java process
> average
> > around 8 - 10 GB. All searchers have final index size of 9 GB. So in
> total
> > there are 9X10 (shards) =  90GB worth of index files.
> >
> >  We have observed the following issue when we trigger indexing . In about
> > 10 minutes after we trigger indexing on 14 parallel hosts, the replicas
> > goes in to recovery mode. This happens to all the shards . In about 20
> > minutes more and more replicas start going into recovery mode. After
> about
> > half an hour all replicas except the leader are in recovery mode. We
> cannot
> > throttle the indexing load as that will increase our overall indexing
> time.
> > So to overcome this issue, we remove all the replicas before we trigger
> the
> > indexing and then add them back after the indexing finishes.
> >
> > We observe the same behavior of replicas going into recovery when we do
> > incremental indexing. We cannot remove replicas during our incremental
> > indexing because it is also serving live traffic. We tried to throttle
> our
> > indexing speed , however the cluster still goes into recovery .
> >
> > If we leave the cluster as it , when the indexing finishes , it
> eventually
> > recovers after a while. As it is serving live traffic we cannot have
> these
> > replicas go into recovery mode because it degrades the search performance
> > also , our tests have shown.
> >
> > We have tried different commit settings like below
> >
> > a) No auto soft commit, no auto hard commit and a commit triggered at the
> > end of indexing b) No auto soft commit, yes auto hard commit and a commit
> > in the end of indexing
> > c) Yes auto soft commit , no auto hard commit
> > d) Yes auto soft commit , yes auto hard commit
> > e) Different frequency setting for commits for above. Please NOTE that we
> > have tried 15 minute soft commit setting and 30 minutes hard commit
> > settings. Same time settings for both, 30 minute soft commit and an hour
> > hard commit setting
> >
> > Unfortunately all the above yields the same behavior . The replicas still
> > goes in recovery We have increased the zookeeper timeout from 30 seconds
> to
> > 5 minutes and the problem persists. Is there any setting that would fix
> > this issue ?
> >
> > --
> > *********************************************
> > Vijay Sekhri
> > *********************************************
> >
>



-- 
*********************************************
Vijay Sekhri
*********************************************

Re: replicas goes in recovery mode right after update

Posted by Erick Erickson <er...@gmail.com>.
Shawn directed you over here to the user list, but I see this note on
SOLR-7030:
"All our searchers have 12 GB of RAM available and have quad core Intel(R)
Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
jboss and solr in it . All 12 GB is available as heap for the java
process..."

So you have 12G physical memory and have allocated 12G to the Java process?
This is an anti-pattern. If that's
the case, your operating system is being starved for memory, probably
hitting a state where it spends all of its
time in stop-the-world garbage collection, eventually it doesn't respond to
Zookeeper's ping so Zookeeper
thinks the node is down and puts it into recovery. Where it spends a lot of
time doing... essentially nothing.

About the hard and soft commits: I suspect these are entirely unrelated,
but here's a blog on what they do, you
should pick the configuration that supports your use case (i.e. how much
latency can you stand between indexing
and being able to search?).

https://lucidworks.com/blog/understanding-transaction-logs-softcommit-and-commit-in-sorlcloud/

Here's one very good reason you shouldn't starve your op system by
allocating all the physical memory to the JVM:
http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html


But your biggest problem is that you have far too much of your physical
memory allocated to the JVM. This
will cause you endless problems, you just need more physical memory on
those boxes. It's _possible_ you could
get by with less memory for the JVM, counterintuitive as it seems try 8G or
maybe even 6G. At some point
you'll hit OOM errors, but that'll give you a lower limit on what the JVM
needs.

Unless I've mis-interpreted what you've written, though, I doubt you'll get
stable with that much memory allocated
to the JVM.

Best,
Erick



On Sun, Jan 25, 2015 at 1:02 PM, Vijay Sekhri <se...@gmail.com> wrote:

> We have a cluster of solr cloud server with 10 shards and 4 replicas in
> each shard in our stress environment. In our prod environment we will have
> 10 shards and 15 replicas in each shard. Our current commit settings are as
> follows
>
> *    <autoSoftCommit>*
> *        <maxDocs>500000</maxDocs>*
> *        <maxTime>180000</maxTime>*
> *    </autoSoftCommit>*
> *    <autoCommit>*
> *        <maxDocs>2000000</maxDocs>*
> *        <maxTime>180000</maxTime>*
> *        <openSearcher>false</openSearcher>*
> *    </autoCommit>*
>
>
> We indexed roughly 90 Million docs. We have two different ways to index
> documents a) Full indexing. It takes 4 hours to index 90 Million docs and
> the rate of docs coming to the searcher is around 6000 per second b)
> Incremental indexing. It takes an hour to indexed delta changes. Roughly
> there are 3 million changes and rate of docs coming to the searchers is
> 2500
> per second
>
> We have two collections search1 and search2. When we do full indexing , we
> do it in search2 collection while search1 is serving live traffic. After it
> finishes we swap the collection using aliases so that the search2
> collection serves live traffic while search1 becomes available for next
> full indexing run. When we do incremental indexing we do it in the search1
> collection which is serving live traffic.
>
> All our searchers have 12 GB of RAM available and have quad core Intel(R)
> Xeon(R) CPU X5570 @ 2.93GHz. There is only one java process running i.e
> jboss and solr in it . All 12 GB is available as heap for the java
> process.  We have observed that the heap memory of the java process average
> around 8 - 10 GB. All searchers have final index size of 9 GB. So in total
> there are 9X10 (shards) =  90GB worth of index files.
>
>  We have observed the following issue when we trigger indexing . In about
> 10 minutes after we trigger indexing on 14 parallel hosts, the replicas
> goes in to recovery mode. This happens to all the shards . In about 20
> minutes more and more replicas start going into recovery mode. After about
> half an hour all replicas except the leader are in recovery mode. We cannot
> throttle the indexing load as that will increase our overall indexing time.
> So to overcome this issue, we remove all the replicas before we trigger the
> indexing and then add them back after the indexing finishes.
>
> We observe the same behavior of replicas going into recovery when we do
> incremental indexing. We cannot remove replicas during our incremental
> indexing because it is also serving live traffic. We tried to throttle our
> indexing speed , however the cluster still goes into recovery .
>
> If we leave the cluster as it , when the indexing finishes , it eventually
> recovers after a while. As it is serving live traffic we cannot have these
> replicas go into recovery mode because it degrades the search performance
> also , our tests have shown.
>
> We have tried different commit settings like below
>
> a) No auto soft commit, no auto hard commit and a commit triggered at the
> end of indexing b) No auto soft commit, yes auto hard commit and a commit
> in the end of indexing
> c) Yes auto soft commit , no auto hard commit
> d) Yes auto soft commit , yes auto hard commit
> e) Different frequency setting for commits for above. Please NOTE that we
> have tried 15 minute soft commit setting and 30 minutes hard commit
> settings. Same time settings for both, 30 minute soft commit and an hour
> hard commit setting
>
> Unfortunately all the above yields the same behavior . The replicas still
> goes in recovery We have increased the zookeeper timeout from 30 seconds to
> 5 minutes and the problem persists. Is there any setting that would fix
> this issue ?
>
> --
> *********************************************
> Vijay Sekhri
> *********************************************
>