What version are you testing? Thought we had addressed this.
-- 
Mark Miller
about.me/markrmiller

On April 16, 2014 at 6:02:09 PM, Jessica Mallet (mewmewb...@gmail.com) wrote:

Hi Furkan,  

Thanks for the reply. I understand the intent. However, in the case I  
described, the follower is blocked on looking for a leader (throws the  
pasted exception because it can't find the leader) before it participates  
in election; therefore, it will never come up while the leader waits for it  
to come up (they're deadlocked waiting for each other). What I'm suggesting  
is that maybe the follower should just just skip waitForLeaderToSeeDownState  
when there's no leader (instead of failing with the pasted stacktrace) and  
go ahead and start participating in election. That way the leader will see  
more replicas come up, and they can sync with each other and move on.  

Thanks,  
Jessica  


On Sat, Apr 12, 2014 at 4:14 PM, Furkan KAMACI <furkankam...@gmail.com>wrote:  

> Hi;  
>  
> There is an explanation as follows: "This is meant to protect the case  
> where you stop a shard or it fails and then the first node to get started  
> back up has stale data - you don't want it to just become the leader. So we  
> wait to see everyone we know about in the shard up to 3 or 5 min by  
> default. Then we know all the shards participate in the leader election and  
> the leader will end up with all updates it should have." You can check it  
> from here:  
>  
> http://mail-archives.apache.org/mod_mbox/lucene-solr-user/201306.mbox/%3ccajt9wng_yykcxggentgcxguhhcjhidear-jygpgrnkaedrz...@mail.gmail.com%3E
>   
>  
> Thanks;  
> Furkan KAMACI  
>  
>  
> 2014-04-08 23:51 GMT+03:00 Jessica Mallet <mewmewb...@gmail.com>:  
>  
> > To clarify, when I said "leader" and "follower" I meant the old leader  
> and  
> > follower before the zookeeper session expiration. When they're recovering  
> > there's no leader.  
> >  
> >  
> > On Tue, Apr 8, 2014 at 1:49 PM, Jessica Mallet <mewmewb...@gmail.com>  
> > wrote:  
> >  
> > > I'm playing with dropping the cluster's connections to zookeeper and  
> then  
> > > reconnecting them, and during recovery, I always see this on the  
> leader's  
> > > logs:  
> > >  
> > > ElectionContext.java (line 361) Waiting until we see more replicas up  
> for  
> > > shard shard1: total=2 found=1 timeoutin=139902  
> > >  
> > > and then on the follower, I see:  
> > > SolrException.java (line 121) There was a problem finding the leader in  
> > > zk:org.apache.solr.common.SolrException: Could not get leader props  
> > > at  
> > >  
> org.apache.solr.cloud.ZkController.getLeaderProps(ZkController.java:958)  
> > > at  
> > >  
> org.apache.solr.cloud.ZkController.getLeaderProps(ZkController.java:922)  
> > > at  
> > >  
> >  
> org.apache.solr.cloud.ZkController.waitForLeaderToSeeDownState(ZkController.java:1463)
>   
> > > at  
> > >  
> >  
> org.apache.solr.cloud.ZkController.registerAllCoresAsDown(ZkController.java:380)
>   
> > > at  
> > > org.apache.solr.cloud.ZkController.access$100(ZkController.java:84)  
> > > at  
> > > org.apache.solr.cloud.ZkController$1.command(ZkController.java:232)  
> > > at  
> > >  
> >  
> org.apache.solr.common.cloud.ConnectionManager$2$1.run(ConnectionManager.java:179)
>   
> > > Caused by: org.apache.zookeeper.KeeperException$NoNodeException:  
> > > KeeperErrorCode = NoNode for /collections/lc4/leaders/shard1  
> > > at  
> > > org.apache.zookeeper.KeeperException.create(KeeperException.java:111)  
> > > at  
> > > org.apache.zookeeper.KeeperException.create(KeeperException.java:51)  
> > > at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1151)  
> > > at  
> > >  
> >  
> org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:273)  
> > > at  
> > >  
> >  
> org.apache.solr.common.cloud.SolrZkClient$7.execute(SolrZkClient.java:270)  
> > > at  
> > >  
> >  
> org.apache.solr.common.cloud.ZkCmdExecutor.retryOperation(ZkCmdExecutor.java:73)
>   
> > > at  
> > >  
> org.apache.solr.common.cloud.SolrZkClient.getData(SolrZkClient.java:270)  
> > > at  
> > >  
> org.apache.solr.cloud.ZkController.getLeaderProps(ZkController.java:936)  
> > > ... 6 more  
> > >  
> > > They block each other's progress until leader decides to give up and  
> not  
> > > wait for more replicas to come up:  
> > >  
> > > ElectionContext.java (line 368) Was waiting for replicas to come up,  
> but  
> > > they are taking too long - assuming they won't come back till later  
> > >  
> > > and then recovery moves forward again.  
> > >  
> > > Should waitForLeaderToSeeDownState move on if there's no leader at the  
> > > moment?  
> > > Thanks,  
> > > Jessica  
> > >  
> >  
>  

Reply via email to