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 Bernd Fehling <be...@uni-bielefeld.de> on 2019/01/08 09:42:14 UTC

Re: REBALANCELEADERS is not reliable

Hi Erick,

after some more hours of debugging the rough result is, who ever invented
this leader election did not check if an action returns the estimated
result. There are only checks for exceptions, true/false, new sequence
numbers and so on, but never if a leader election to the preferredleader
really took place.

If doing a rebalanceleaders to preferredleader I also have to check if:
- a rebalance took place
- the preferredleader has really become leader (and not anyone else)

Currently this is not checked and the call rebalanceleaders to preferredleader
is like a shot into the dark with hope of success. And thats why any
problems have never been discovered or reported.

Bernd


Am 21.12.18 um 18:00 schrieb Erick Erickson:
> I looked at the test last night and it's...disturbing. It succeeds
> 100% of the time. Manual testing seems to fail very often.
> Of course it was late and I was a bit cross-eyed, so maybe
> I wasn't looking at the manual tests correctly. Or maybe the
> test is buggy.
> 
> I beasted the test 100x last night and all of them succeeded.
> 
> This was with all NRT replicas.
> 
> Today I'm going to modify the test into a stand-alone program
> to see if it's something in the test environment that causes
> it to succeed. I've got to get this to fail as a unit test before I
> have confidence in any fixes, and also confidence that things
> like this will be caught going forward.
> 
> Erick
> 
> On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
> <be...@uni-bielefeld.de> wrote:
>>
>> As far as I could see with debugger there is still a problem in requeing.
>>
>> There is a watcher and it is recognized that the watcher is not a preferredleader.
>> So it tries to locate a preferredleader with success.
>> It then calls makeReplicaFirstWatcher and gets a new sequence number for
>> the preferredleader replica. But now we have two replicas with the same
>> sequence number. One replica which already owns that sequence number and
>> the replica which got the new (and the same) number as new sequence number.
>> It now tries to solve this with queueNodesWithSameSequence.
>> Might be something in rejoinElection.
>> At least the call to rejoinElection seems right. For preferredleader it
>> is true for rejoinAtHead and for the other replica with same sequence number
>> it is false for rejoinAtHead.
>>
>> A test case should have 3 shards with 3 cores per shard and should try to
>> set preferredleader to different replicas at random. And then try to
>> rebalance and check the results.
>>
>> So far, regards, Bernd
>>
>>
>> Am 21.12.18 um 07:11 schrieb Erick Erickson:
>>> I'm reworking the test case, so hold off on doing that. If you want to
>>> raise a JIRA, though. please do and attach your patch...
>>>
>>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
>>>>
>>>> Nothing that I know of was _intentionally_ changed with this between
>>>> 6x and 7x. That said, nothing that I know of was done to verify that
>>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
>>>> test "TestRebalanceLeaders" for this functionality that has run since
>>>> the feature was put in, but it has _not_ been modified to create TLOG
>>>> and PULL replicas and test with those.
>>>>
>>>> For this patch to be complete, we should either extend that test or
>>>> make another that fails without this patch and succeeds with it.
>>>>
>>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
>>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
>>>> various cases.
>>>>
>>>> Best,
>>>> Erick
>>>>
>>>>
>>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
>>>> <be...@uni-bielefeld.de> wrote:
>>>>>
>>>>> Hi Vadim,
>>>>> I just tried it with 6.6.5.
>>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
>>>>> one shard to become leader. But noticed that one shard already was
>>>>> leader. No errors or exceptions in logs.
>>>>> May be I should enable debug logging and try again to see all logging
>>>>> messages from the patch.
>>>>>
>>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
>>>>> it works for you.
>>>>>
>>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
>>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
>>>>>
>>>>> Regards, Bernd
>>>>>
>>>>>
>>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
>>>>>> Yes! It works!
>>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
>>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
>>>>>> Original patch was slightly incompatible with 7.6.0
>>>>>> I hope this patch will help to try this feature with 7.6
>>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
>>>>>>
>>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
>>>>>> But new replica types made it very helpful to keep big clusters in order...
>>>>>>
>>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
>>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
>>>>>>

Re: REBALANCELEADERS is not reliable

Posted by Bernd Fehling <be...@uni-bielefeld.de>.
Hi Erik,

patches and the new comments look good.
Unfortunately I'm at 6.6.5 and can't test this with my cloud.
Replica (o.a.s.common.cloud.Replica) at 6.6.5 is to far away from 7.6 and up.
And a backport for 6.6.5 is to much rework, if possible at all.

Thanks for solving this issue.

Regards,
Bernd


Am 20.01.19 um 17:04 schrieb Erick Erickson:
> Bernd:
> 
> I just committed fixes on SOLR-13091 and SOLR-10935 to the repo, if
> you wanted to give it a whirl it's ready. By tonight (Sunday) I expect
> to change the response format a bit and update the ref guide, although
> you'll have to look at the doc changes in the format. There's a new
> summary section that gives "Success" or "Failure" that's supposed to
> be the only thing you really need to check...
> 
> One judgement call I made was that if a replica on a down node is the
> preferredLeader, it _can't_ be made leader, but this is still labeled
> "Success".
> 
> Best,
> Erick
> 
> On Sun, Jan 13, 2019 at 7:43 PM Erick Erickson <er...@gmail.com> wrote:
>>
>> Bernd:
>>
>> I just attached a patch to
>> https://issues.apache.org/jira/browse/SOLR-13091. It's still rough,
>> the response from REBALANCELEADERS needs quite a bit of work (lots of
>> extra stuff in it now, and no overall verification).
>> I haven't run all the tests, nor precommit.
>>
>> I wanted to get something up so if you have a test environment that
>> you can easily test it in you'd have an early chance to play with it.
>>
>> It's against master, I also haven't tried to backport to 8.0 or 7x
>> yet. I doubt it'll be a problem, but if it does't apply cleanly let me
>> know.
>>
>> Best,
>> Erick
>>
>> On Fri, Jan 11, 2019 at 8:33 AM Erick Erickson <er...@gmail.com> wrote:
>>>
>>> bq: You have to check if the cores, participating in leadership
>>> election, are _really_
>>> in sync. And this must be done before starting any rebalance.
>>> Sounds ugly... :-(
>>>
>>> This _should_ not be necessary. I'll add parenthetically that leader
>>> election has
>>> been extensively re-worked in Solr 7.3+ though because "interesting" things
>>> could happen.
>>>
>>> Manipulating the leader election queue is really no different than
>>> having to deal with, say, someone killing the leader un-gracefully. It  should
>>> "just work". That said if you're seeing evidence to the contrary that's reality.
>>>
>>> What do you mean by "stats" though? It's perfectly ordinary for there to
>>> be different numbers of _deleted_ documents on various replicas, and
>>> consequently things like term frequencies and doc frequencies being
>>> different. What's emphatically _not_ expected is for there to be different
>>> numbers of "live" docs.
>>>
>>> "making sure nodes are in sync" is certainly an option. That should all
>>> be automatic if you pause indexing and issue a commit, _then_
>>> do a rebalance.
>>>
>>> I certainly agree that the code is broken and needs to be fixed, but I
>>> also have to ask how many shards are we talking here? The code was
>>> originally written for the case where 100s of leaders could be on the
>>> same node, until you get in to a significant number of leaders on
>>> a single node (10s at least) there haven't been reliable stats showing
>>> that it's a performance issue. If you have threshold numbers where
>>> you've seen it make a material difference it'd be great to share them.
>>>
>>> And I won't be getting back to this until the weekend, other urgent
>>> stuff has come up...
>>>
>>> Best,
>>> Erick
>>>
>>> On Fri, Jan 11, 2019 at 12:58 AM Bernd Fehling
>>> <be...@uni-bielefeld.de> wrote:
>>>>
>>>> Hi Erik,
>>>> yes, I would be happy to test any patches.
>>>>
>>>> Good news, I got rebalance working.
>>>> After running the rebalance about 50 times with debugger and watching
>>>> the behavior of my problem shard and its core_nodes within my test cloud
>>>> I came to the point of failure. I solved it and now it works.
>>>>
>>>> Bad news, rebalance is still not reliable and there are many more
>>>> problems and point of failure initiated by rebalanceLeaders or better
>>>> by re-queueing the watchlist.
>>>>
>>>> How I located _my_ problem:
>>>> Test cloud is 5 server (VM), 5 shards, 3 replica per shard, 1 java
>>>> instance per server. 3 separate zookeepers.
>>>> My problem, shard2 wasn't willing to rebalance to a specific core_node.
>>>> core_nodes related (core_node1, core_node2, core_node10).
>>>> core_node10 was the preferredLeader.
>>>> It was just changing leader ship between core_node1 and core_node2,
>>>> back and forth, whenever I called rebalanceLeader.
>>>> First step, I stopped the server holding core_node2.
>>>> Result, the leadership was staying at core_node1 whenever I called rebalanceLeaders.
>>>> Second step, from debugger I _forced_ during rebalanceLeaders the
>>>> system to give the leadership to core_node10.
>>>> Result, there was no leader anymore for that shard. Yes it can happen,
>>>> you can end up with a shard having no leader but active core_nodes!!!
>>>> To fix this I was giving preferredLeader to core_node1 and called rebalanceLeaders.
>>>> After that, preferredLeader was set back to core_node10 and I was back
>>>> at the point I started, all calls to rebalanceLeaders kept the leader at core_node1.
>>>>
>>>>   From the debug logs I got the hint about PeerSync of cores and IndexFingerprint.
>>>> The stats from my problem core_node10 showed that they differ from leader core_node1.
>>>> And the system notices the difference, starts a PeerSync and ends with success.
>>>> But actually the PeerSync seem to fail, because the stats of core_node1 and
>>>> core_node10 still differ afterwards.
>>>> Solution, I also stopped my server holding my problem core_node10, wiped all data
>>>> directories and started that server again. The core_nodes where rebuilt from leader
>>>> and now they are really in sync.
>>>> Calling now rebalanceLeaders ended now with success to preferredLeader.
>>>>
>>>> My guess:
>>>> You have to check if the cores, participating in leadership election, are _really_
>>>> in sync. And this must be done before starting any rebalance.
>>>> Sounds ugly... :-(
>>>>
>>>> Next question, why is PeerSync not reporting an error?
>>>> There is an info about "PeerSync START", "PeerSync Received 0 versions from ... fingeprint:null"
>>>> and "PeerSync DONE. sync succeeded" but the cores are not really in sync.
>>>>
>>>> Another test I did (with my new knowledge about synced cores):
>>>> - Removing all preferredLeader properties
>>>> - stopping, wiping data directory, starting all server one by one to get
>>>>     all cores of all shards in sync
>>>> - setting one preferredLeader for each shard but different from the actual leader
>>>> - calling rebalanceLeaders succeeded only at 2 shards with the first run,
>>>>     not for all 5 shards (even with really all cores in sync).
>>>> - after calling rebalanceLeaders again the other shards succeeded also.
>>>> Result, rebalanceLeaders is still not reliable.
>>>>
>>>> I have to mention that I have about 520.000 docs per core in my test cloud
>>>> and that there might also be a timing issue between calling rebalanceLeaders,
>>>> detecting that cores to become leader are not in sync with actual leader,
>>>> and resync while waiting for new leader election.
>>>>
>>>> So far,
>>>> Bernd
>>>>
>>>>
>>>> Am 10.01.19 um 17:02 schrieb Erick Erickson:
>>>>> Bernd:
>>>>>
>>>>> Don't feel bad about missing it, I wrote the silly stuff and it took me
>>>>> some time to remember.....
>>>>>
>>>>> Those are  the rules.
>>>>>
>>>>> It's always humbling to look back at my own code and say "that
>>>>> idiot should have put some comments in here..." ;)
>>>>>
>>>>> yeah, I agree there are a lot of moving parts here. I have a note to
>>>>> myself to provide better feedback in the response. You're absolutely
>>>>> right that we fire all these commands and hope they all work.  Just
>>>>> returning "success" status doesn't guarantee leadership change.
>>>>>
>>>>> I'll be on another task the rest of this week, but I should be able
>>>>> to dress things up over the weekend. That'll give you a patch to test
>>>>> if you're willing.
>>>>>
>>>>> The actual code changes are pretty minimal, the bulk of the patch
>>>>> will be the reworked test.
>>>>>
>>>>> Best,
>>>>> Erick
>>>>>

-- 
*************************************************************
Bernd Fehling                    Bielefeld University Library
Dipl.-Inform. (FH)                LibTec - Library Technology
Universitätsstr. 25                  and Knowledge Management
33615 Bielefeld
Tel. +49 521 106-4060       bernd.fehling(at)uni-bielefeld.de
           https://www.ub.uni-bielefeld.de/~befehl/

BASE - Bielefeld Academic Search Engine - www.base-search.net
*************************************************************

Re: REBALANCELEADERS is not reliable

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

I just committed fixes on SOLR-13091 and SOLR-10935 to the repo, if
you wanted to give it a whirl it's ready. By tonight (Sunday) I expect
to change the response format a bit and update the ref guide, although
you'll have to look at the doc changes in the format. There's a new
summary section that gives "Success" or "Failure" that's supposed to
be the only thing you really need to check...

One judgement call I made was that if a replica on a down node is the
preferredLeader, it _can't_ be made leader, but this is still labeled
"Success".

Best,
Erick

On Sun, Jan 13, 2019 at 7:43 PM Erick Erickson <er...@gmail.com> wrote:
>
> Bernd:
>
> I just attached a patch to
> https://issues.apache.org/jira/browse/SOLR-13091. It's still rough,
> the response from REBALANCELEADERS needs quite a bit of work (lots of
> extra stuff in it now, and no overall verification).
> I haven't run all the tests, nor precommit.
>
> I wanted to get something up so if you have a test environment that
> you can easily test it in you'd have an early chance to play with it.
>
> It's against master, I also haven't tried to backport to 8.0 or 7x
> yet. I doubt it'll be a problem, but if it does't apply cleanly let me
> know.
>
> Best,
> Erick
>
> On Fri, Jan 11, 2019 at 8:33 AM Erick Erickson <er...@gmail.com> wrote:
> >
> > bq: You have to check if the cores, participating in leadership
> > election, are _really_
> > in sync. And this must be done before starting any rebalance.
> > Sounds ugly... :-(
> >
> > This _should_ not be necessary. I'll add parenthetically that leader
> > election has
> > been extensively re-worked in Solr 7.3+ though because "interesting" things
> > could happen.
> >
> > Manipulating the leader election queue is really no different than
> > having to deal with, say, someone killing the leader un-gracefully. It  should
> > "just work". That said if you're seeing evidence to the contrary that's reality.
> >
> > What do you mean by "stats" though? It's perfectly ordinary for there to
> > be different numbers of _deleted_ documents on various replicas, and
> > consequently things like term frequencies and doc frequencies being
> > different. What's emphatically _not_ expected is for there to be different
> > numbers of "live" docs.
> >
> > "making sure nodes are in sync" is certainly an option. That should all
> > be automatic if you pause indexing and issue a commit, _then_
> > do a rebalance.
> >
> > I certainly agree that the code is broken and needs to be fixed, but I
> > also have to ask how many shards are we talking here? The code was
> > originally written for the case where 100s of leaders could be on the
> > same node, until you get in to a significant number of leaders on
> > a single node (10s at least) there haven't been reliable stats showing
> > that it's a performance issue. If you have threshold numbers where
> > you've seen it make a material difference it'd be great to share them.
> >
> > And I won't be getting back to this until the weekend, other urgent
> > stuff has come up...
> >
> > Best,
> > Erick
> >
> > On Fri, Jan 11, 2019 at 12:58 AM Bernd Fehling
> > <be...@uni-bielefeld.de> wrote:
> > >
> > > Hi Erik,
> > > yes, I would be happy to test any patches.
> > >
> > > Good news, I got rebalance working.
> > > After running the rebalance about 50 times with debugger and watching
> > > the behavior of my problem shard and its core_nodes within my test cloud
> > > I came to the point of failure. I solved it and now it works.
> > >
> > > Bad news, rebalance is still not reliable and there are many more
> > > problems and point of failure initiated by rebalanceLeaders or better
> > > by re-queueing the watchlist.
> > >
> > > How I located _my_ problem:
> > > Test cloud is 5 server (VM), 5 shards, 3 replica per shard, 1 java
> > > instance per server. 3 separate zookeepers.
> > > My problem, shard2 wasn't willing to rebalance to a specific core_node.
> > > core_nodes related (core_node1, core_node2, core_node10).
> > > core_node10 was the preferredLeader.
> > > It was just changing leader ship between core_node1 and core_node2,
> > > back and forth, whenever I called rebalanceLeader.
> > > First step, I stopped the server holding core_node2.
> > > Result, the leadership was staying at core_node1 whenever I called rebalanceLeaders.
> > > Second step, from debugger I _forced_ during rebalanceLeaders the
> > > system to give the leadership to core_node10.
> > > Result, there was no leader anymore for that shard. Yes it can happen,
> > > you can end up with a shard having no leader but active core_nodes!!!
> > > To fix this I was giving preferredLeader to core_node1 and called rebalanceLeaders.
> > > After that, preferredLeader was set back to core_node10 and I was back
> > > at the point I started, all calls to rebalanceLeaders kept the leader at core_node1.
> > >
> > >  From the debug logs I got the hint about PeerSync of cores and IndexFingerprint.
> > > The stats from my problem core_node10 showed that they differ from leader core_node1.
> > > And the system notices the difference, starts a PeerSync and ends with success.
> > > But actually the PeerSync seem to fail, because the stats of core_node1 and
> > > core_node10 still differ afterwards.
> > > Solution, I also stopped my server holding my problem core_node10, wiped all data
> > > directories and started that server again. The core_nodes where rebuilt from leader
> > > and now they are really in sync.
> > > Calling now rebalanceLeaders ended now with success to preferredLeader.
> > >
> > > My guess:
> > > You have to check if the cores, participating in leadership election, are _really_
> > > in sync. And this must be done before starting any rebalance.
> > > Sounds ugly... :-(
> > >
> > > Next question, why is PeerSync not reporting an error?
> > > There is an info about "PeerSync START", "PeerSync Received 0 versions from ... fingeprint:null"
> > > and "PeerSync DONE. sync succeeded" but the cores are not really in sync.
> > >
> > > Another test I did (with my new knowledge about synced cores):
> > > - Removing all preferredLeader properties
> > > - stopping, wiping data directory, starting all server one by one to get
> > >    all cores of all shards in sync
> > > - setting one preferredLeader for each shard but different from the actual leader
> > > - calling rebalanceLeaders succeeded only at 2 shards with the first run,
> > >    not for all 5 shards (even with really all cores in sync).
> > > - after calling rebalanceLeaders again the other shards succeeded also.
> > > Result, rebalanceLeaders is still not reliable.
> > >
> > > I have to mention that I have about 520.000 docs per core in my test cloud
> > > and that there might also be a timing issue between calling rebalanceLeaders,
> > > detecting that cores to become leader are not in sync with actual leader,
> > > and resync while waiting for new leader election.
> > >
> > > So far,
> > > Bernd
> > >
> > >
> > > Am 10.01.19 um 17:02 schrieb Erick Erickson:
> > > > Bernd:
> > > >
> > > > Don't feel bad about missing it, I wrote the silly stuff and it took me
> > > > some time to remember.....
> > > >
> > > > Those are  the rules.
> > > >
> > > > It's always humbling to look back at my own code and say "that
> > > > idiot should have put some comments in here..." ;)
> > > >
> > > > yeah, I agree there are a lot of moving parts here. I have a note to
> > > > myself to provide better feedback in the response. You're absolutely
> > > > right that we fire all these commands and hope they all work.  Just
> > > > returning "success" status doesn't guarantee leadership change.
> > > >
> > > > I'll be on another task the rest of this week, but I should be able
> > > > to dress things up over the weekend. That'll give you a patch to test
> > > > if you're willing.
> > > >
> > > > The actual code changes are pretty minimal, the bulk of the patch
> > > > will be the reworked test.
> > > >
> > > > Best,
> > > > Erick
> > > >

Re: REBALANCELEADERS is not reliable

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

I just attached a patch to
https://issues.apache.org/jira/browse/SOLR-13091. It's still rough,
the response from REBALANCELEADERS needs quite a bit of work (lots of
extra stuff in it now, and no overall verification).
I haven't run all the tests, nor precommit.

I wanted to get something up so if you have a test environment that
you can easily test it in you'd have an early chance to play with it.

It's against master, I also haven't tried to backport to 8.0 or 7x
yet. I doubt it'll be a problem, but if it does't apply cleanly let me
know.

Best,
Erick

On Fri, Jan 11, 2019 at 8:33 AM Erick Erickson <er...@gmail.com> wrote:
>
> bq: You have to check if the cores, participating in leadership
> election, are _really_
> in sync. And this must be done before starting any rebalance.
> Sounds ugly... :-(
>
> This _should_ not be necessary. I'll add parenthetically that leader
> election has
> been extensively re-worked in Solr 7.3+ though because "interesting" things
> could happen.
>
> Manipulating the leader election queue is really no different than
> having to deal with, say, someone killing the leader un-gracefully. It  should
> "just work". That said if you're seeing evidence to the contrary that's reality.
>
> What do you mean by "stats" though? It's perfectly ordinary for there to
> be different numbers of _deleted_ documents on various replicas, and
> consequently things like term frequencies and doc frequencies being
> different. What's emphatically _not_ expected is for there to be different
> numbers of "live" docs.
>
> "making sure nodes are in sync" is certainly an option. That should all
> be automatic if you pause indexing and issue a commit, _then_
> do a rebalance.
>
> I certainly agree that the code is broken and needs to be fixed, but I
> also have to ask how many shards are we talking here? The code was
> originally written for the case where 100s of leaders could be on the
> same node, until you get in to a significant number of leaders on
> a single node (10s at least) there haven't been reliable stats showing
> that it's a performance issue. If you have threshold numbers where
> you've seen it make a material difference it'd be great to share them.
>
> And I won't be getting back to this until the weekend, other urgent
> stuff has come up...
>
> Best,
> Erick
>
> On Fri, Jan 11, 2019 at 12:58 AM Bernd Fehling
> <be...@uni-bielefeld.de> wrote:
> >
> > Hi Erik,
> > yes, I would be happy to test any patches.
> >
> > Good news, I got rebalance working.
> > After running the rebalance about 50 times with debugger and watching
> > the behavior of my problem shard and its core_nodes within my test cloud
> > I came to the point of failure. I solved it and now it works.
> >
> > Bad news, rebalance is still not reliable and there are many more
> > problems and point of failure initiated by rebalanceLeaders or better
> > by re-queueing the watchlist.
> >
> > How I located _my_ problem:
> > Test cloud is 5 server (VM), 5 shards, 3 replica per shard, 1 java
> > instance per server. 3 separate zookeepers.
> > My problem, shard2 wasn't willing to rebalance to a specific core_node.
> > core_nodes related (core_node1, core_node2, core_node10).
> > core_node10 was the preferredLeader.
> > It was just changing leader ship between core_node1 and core_node2,
> > back and forth, whenever I called rebalanceLeader.
> > First step, I stopped the server holding core_node2.
> > Result, the leadership was staying at core_node1 whenever I called rebalanceLeaders.
> > Second step, from debugger I _forced_ during rebalanceLeaders the
> > system to give the leadership to core_node10.
> > Result, there was no leader anymore for that shard. Yes it can happen,
> > you can end up with a shard having no leader but active core_nodes!!!
> > To fix this I was giving preferredLeader to core_node1 and called rebalanceLeaders.
> > After that, preferredLeader was set back to core_node10 and I was back
> > at the point I started, all calls to rebalanceLeaders kept the leader at core_node1.
> >
> >  From the debug logs I got the hint about PeerSync of cores and IndexFingerprint.
> > The stats from my problem core_node10 showed that they differ from leader core_node1.
> > And the system notices the difference, starts a PeerSync and ends with success.
> > But actually the PeerSync seem to fail, because the stats of core_node1 and
> > core_node10 still differ afterwards.
> > Solution, I also stopped my server holding my problem core_node10, wiped all data
> > directories and started that server again. The core_nodes where rebuilt from leader
> > and now they are really in sync.
> > Calling now rebalanceLeaders ended now with success to preferredLeader.
> >
> > My guess:
> > You have to check if the cores, participating in leadership election, are _really_
> > in sync. And this must be done before starting any rebalance.
> > Sounds ugly... :-(
> >
> > Next question, why is PeerSync not reporting an error?
> > There is an info about "PeerSync START", "PeerSync Received 0 versions from ... fingeprint:null"
> > and "PeerSync DONE. sync succeeded" but the cores are not really in sync.
> >
> > Another test I did (with my new knowledge about synced cores):
> > - Removing all preferredLeader properties
> > - stopping, wiping data directory, starting all server one by one to get
> >    all cores of all shards in sync
> > - setting one preferredLeader for each shard but different from the actual leader
> > - calling rebalanceLeaders succeeded only at 2 shards with the first run,
> >    not for all 5 shards (even with really all cores in sync).
> > - after calling rebalanceLeaders again the other shards succeeded also.
> > Result, rebalanceLeaders is still not reliable.
> >
> > I have to mention that I have about 520.000 docs per core in my test cloud
> > and that there might also be a timing issue between calling rebalanceLeaders,
> > detecting that cores to become leader are not in sync with actual leader,
> > and resync while waiting for new leader election.
> >
> > So far,
> > Bernd
> >
> >
> > Am 10.01.19 um 17:02 schrieb Erick Erickson:
> > > Bernd:
> > >
> > > Don't feel bad about missing it, I wrote the silly stuff and it took me
> > > some time to remember.....
> > >
> > > Those are  the rules.
> > >
> > > It's always humbling to look back at my own code and say "that
> > > idiot should have put some comments in here..." ;)
> > >
> > > yeah, I agree there are a lot of moving parts here. I have a note to
> > > myself to provide better feedback in the response. You're absolutely
> > > right that we fire all these commands and hope they all work.  Just
> > > returning "success" status doesn't guarantee leadership change.
> > >
> > > I'll be on another task the rest of this week, but I should be able
> > > to dress things up over the weekend. That'll give you a patch to test
> > > if you're willing.
> > >
> > > The actual code changes are pretty minimal, the bulk of the patch
> > > will be the reworked test.
> > >
> > > Best,
> > > Erick
> > >

Re: REBALANCELEADERS is not reliable

Posted by Erick Erickson <er...@gmail.com>.
bq: You have to check if the cores, participating in leadership
election, are _really_
in sync. And this must be done before starting any rebalance.
Sounds ugly... :-(

This _should_ not be necessary. I'll add parenthetically that leader
election has
been extensively re-worked in Solr 7.3+ though because "interesting" things
could happen.

Manipulating the leader election queue is really no different than
having to deal with, say, someone killing the leader un-gracefully. It  should
"just work". That said if you're seeing evidence to the contrary that's reality.

What do you mean by "stats" though? It's perfectly ordinary for there to
be different numbers of _deleted_ documents on various replicas, and
consequently things like term frequencies and doc frequencies being
different. What's emphatically _not_ expected is for there to be different
numbers of "live" docs.

"making sure nodes are in sync" is certainly an option. That should all
be automatic if you pause indexing and issue a commit, _then_
do a rebalance.

I certainly agree that the code is broken and needs to be fixed, but I
also have to ask how many shards are we talking here? The code was
originally written for the case where 100s of leaders could be on the
same node, until you get in to a significant number of leaders on
a single node (10s at least) there haven't been reliable stats showing
that it's a performance issue. If you have threshold numbers where
you've seen it make a material difference it'd be great to share them.

And I won't be getting back to this until the weekend, other urgent
stuff has come up...

Best,
Erick

On Fri, Jan 11, 2019 at 12:58 AM Bernd Fehling
<be...@uni-bielefeld.de> wrote:
>
> Hi Erik,
> yes, I would be happy to test any patches.
>
> Good news, I got rebalance working.
> After running the rebalance about 50 times with debugger and watching
> the behavior of my problem shard and its core_nodes within my test cloud
> I came to the point of failure. I solved it and now it works.
>
> Bad news, rebalance is still not reliable and there are many more
> problems and point of failure initiated by rebalanceLeaders or better
> by re-queueing the watchlist.
>
> How I located _my_ problem:
> Test cloud is 5 server (VM), 5 shards, 3 replica per shard, 1 java
> instance per server. 3 separate zookeepers.
> My problem, shard2 wasn't willing to rebalance to a specific core_node.
> core_nodes related (core_node1, core_node2, core_node10).
> core_node10 was the preferredLeader.
> It was just changing leader ship between core_node1 and core_node2,
> back and forth, whenever I called rebalanceLeader.
> First step, I stopped the server holding core_node2.
> Result, the leadership was staying at core_node1 whenever I called rebalanceLeaders.
> Second step, from debugger I _forced_ during rebalanceLeaders the
> system to give the leadership to core_node10.
> Result, there was no leader anymore for that shard. Yes it can happen,
> you can end up with a shard having no leader but active core_nodes!!!
> To fix this I was giving preferredLeader to core_node1 and called rebalanceLeaders.
> After that, preferredLeader was set back to core_node10 and I was back
> at the point I started, all calls to rebalanceLeaders kept the leader at core_node1.
>
>  From the debug logs I got the hint about PeerSync of cores and IndexFingerprint.
> The stats from my problem core_node10 showed that they differ from leader core_node1.
> And the system notices the difference, starts a PeerSync and ends with success.
> But actually the PeerSync seem to fail, because the stats of core_node1 and
> core_node10 still differ afterwards.
> Solution, I also stopped my server holding my problem core_node10, wiped all data
> directories and started that server again. The core_nodes where rebuilt from leader
> and now they are really in sync.
> Calling now rebalanceLeaders ended now with success to preferredLeader.
>
> My guess:
> You have to check if the cores, participating in leadership election, are _really_
> in sync. And this must be done before starting any rebalance.
> Sounds ugly... :-(
>
> Next question, why is PeerSync not reporting an error?
> There is an info about "PeerSync START", "PeerSync Received 0 versions from ... fingeprint:null"
> and "PeerSync DONE. sync succeeded" but the cores are not really in sync.
>
> Another test I did (with my new knowledge about synced cores):
> - Removing all preferredLeader properties
> - stopping, wiping data directory, starting all server one by one to get
>    all cores of all shards in sync
> - setting one preferredLeader for each shard but different from the actual leader
> - calling rebalanceLeaders succeeded only at 2 shards with the first run,
>    not for all 5 shards (even with really all cores in sync).
> - after calling rebalanceLeaders again the other shards succeeded also.
> Result, rebalanceLeaders is still not reliable.
>
> I have to mention that I have about 520.000 docs per core in my test cloud
> and that there might also be a timing issue between calling rebalanceLeaders,
> detecting that cores to become leader are not in sync with actual leader,
> and resync while waiting for new leader election.
>
> So far,
> Bernd
>
>
> Am 10.01.19 um 17:02 schrieb Erick Erickson:
> > Bernd:
> >
> > Don't feel bad about missing it, I wrote the silly stuff and it took me
> > some time to remember.....
> >
> > Those are  the rules.
> >
> > It's always humbling to look back at my own code and say "that
> > idiot should have put some comments in here..." ;)
> >
> > yeah, I agree there are a lot of moving parts here. I have a note to
> > myself to provide better feedback in the response. You're absolutely
> > right that we fire all these commands and hope they all work.  Just
> > returning "success" status doesn't guarantee leadership change.
> >
> > I'll be on another task the rest of this week, but I should be able
> > to dress things up over the weekend. That'll give you a patch to test
> > if you're willing.
> >
> > The actual code changes are pretty minimal, the bulk of the patch
> > will be the reworked test.
> >
> > Best,
> > Erick
> >

Re: REBALANCELEADERS is not reliable

Posted by Bernd Fehling <be...@uni-bielefeld.de>.
Hi Erik,
yes, I would be happy to test any patches.

Good news, I got rebalance working.
After running the rebalance about 50 times with debugger and watching
the behavior of my problem shard and its core_nodes within my test cloud
I came to the point of failure. I solved it and now it works.

Bad news, rebalance is still not reliable and there are many more
problems and point of failure initiated by rebalanceLeaders or better
by re-queueing the watchlist.

How I located _my_ problem:
Test cloud is 5 server (VM), 5 shards, 3 replica per shard, 1 java
instance per server. 3 separate zookeepers.
My problem, shard2 wasn't willing to rebalance to a specific core_node.
core_nodes related (core_node1, core_node2, core_node10).
core_node10 was the preferredLeader.
It was just changing leader ship between core_node1 and core_node2,
back and forth, whenever I called rebalanceLeader.
First step, I stopped the server holding core_node2.
Result, the leadership was staying at core_node1 whenever I called rebalanceLeaders.
Second step, from debugger I _forced_ during rebalanceLeaders the
system to give the leadership to core_node10.
Result, there was no leader anymore for that shard. Yes it can happen,
you can end up with a shard having no leader but active core_nodes!!!
To fix this I was giving preferredLeader to core_node1 and called rebalanceLeaders.
After that, preferredLeader was set back to core_node10 and I was back
at the point I started, all calls to rebalanceLeaders kept the leader at core_node1.

 From the debug logs I got the hint about PeerSync of cores and IndexFingerprint.
The stats from my problem core_node10 showed that they differ from leader core_node1.
And the system notices the difference, starts a PeerSync and ends with success.
But actually the PeerSync seem to fail, because the stats of core_node1 and
core_node10 still differ afterwards.
Solution, I also stopped my server holding my problem core_node10, wiped all data
directories and started that server again. The core_nodes where rebuilt from leader
and now they are really in sync.
Calling now rebalanceLeaders ended now with success to preferredLeader.

My guess:
You have to check if the cores, participating in leadership election, are _really_
in sync. And this must be done before starting any rebalance.
Sounds ugly... :-(

Next question, why is PeerSync not reporting an error?
There is an info about "PeerSync START", "PeerSync Received 0 versions from ... fingeprint:null"
and "PeerSync DONE. sync succeeded" but the cores are not really in sync.

Another test I did (with my new knowledge about synced cores):
- Removing all preferredLeader properties
- stopping, wiping data directory, starting all server one by one to get
   all cores of all shards in sync
- setting one preferredLeader for each shard but different from the actual leader
- calling rebalanceLeaders succeeded only at 2 shards with the first run,
   not for all 5 shards (even with really all cores in sync).
- after calling rebalanceLeaders again the other shards succeeded also.
Result, rebalanceLeaders is still not reliable.

I have to mention that I have about 520.000 docs per core in my test cloud
and that there might also be a timing issue between calling rebalanceLeaders,
detecting that cores to become leader are not in sync with actual leader,
and resync while waiting for new leader election.

So far,
Bernd


Am 10.01.19 um 17:02 schrieb Erick Erickson:
> Bernd:
> 
> Don't feel bad about missing it, I wrote the silly stuff and it took me
> some time to remember.....
> 
> Those are  the rules.
> 
> It's always humbling to look back at my own code and say "that
> idiot should have put some comments in here..." ;)
> 
> yeah, I agree there are a lot of moving parts here. I have a note to
> myself to provide better feedback in the response. You're absolutely
> right that we fire all these commands and hope they all work.  Just
> returning "success" status doesn't guarantee leadership change.
> 
> I'll be on another task the rest of this week, but I should be able
> to dress things up over the weekend. That'll give you a patch to test
> if you're willing.
> 
> The actual code changes are pretty minimal, the bulk of the patch
> will be the reworked test.
> 
> Best,
> Erick
> 

Re: REBALANCELEADERS is not reliable

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

Don't feel bad about missing it, I wrote the silly stuff and it took me
some time to remember.....

Those are  the rules.

It's always humbling to look back at my own code and say "that
idiot should have put some comments in here..." ;)

yeah, I agree there are a lot of moving parts here. I have a note to
myself to provide better feedback in the response. You're absolutely
right that we fire all these commands and hope they all work.  Just
returning "success" status doesn't guarantee leadership change.

I'll be on another task the rest of this week, but I should be able
to dress things up over the weekend. That'll give you a patch to test
if you're willing.

The actual code changes are pretty minimal, the bulk of the patch
will be the reworked test.

Best,
Erick

On Wed, Jan 9, 2019 at 11:59 PM Bernd Fehling
<be...@uni-bielefeld.de> wrote:
>
> Hi Erik,
>
> that is very valuable info I missed.
> Shouldn't that belong into an issue about rework at REBALANCELEADERS?
>
> With your explanation the use of a queue makes sense and now I see some of
> the logic behind.
> - there is the leader and the firstWatcher
> - if firstWatcher goes down or is inactive that one should positioned
>    at end of queue which automatically gives the baton to the next in queue
> - preferredLeader has precedence if state=active
> - there can be only one preferredLeader per shard
>
> Any  more rules?
>
>  From debugging I know there are a lot of actions taken behind the scene
> and many things can go wrong. Parts included are ZkStateReader, Overseer,
> ShardLeaderElectionContextBase, ShardLeaderElectionContext, ElectionContext,
> LeaderElector, some Requests and a HttpSolrCall to REJOINLEADERELECTION.
>
> An info I see when rebalance goes wrong is from ShardLeaderElectionContextBase:
> "No version found for ephemeral leader parent node, won't remove previous leader registration."
>
> Regards, Bernd
>
>
> Am 09.01.19 um 17:22 schrieb Erick Erickson:
> > Executive summary:
> >
> > The central problem is "how can I insert an ephemeral node
> > in a specific place in a ZK queue". The code could be much,
> > much simpler if there were a reliable way to do just that. I haven't
> > looked at more recent ZKs to see if it's possible, I'd love it if
> > there were a better way.
> >
> > On to details:
> >
> > bq.  wonder if we can discover the "inventor" of all this and ask him
> > how it should work
> >
> > Yeah, I can contact that clown. That would be me ;)
> >
> > The way leader election works is a  ZK recipe where each
> > ephemeral node only watches the one in front of it. When a
> > node is deleted, the one watching it is notified.
> >
> > So let's say we have nodes in this order: 1 watched by 2
> > watched by 3... In this case 1 is the leader. If 2 should
> > disappear, 3 gets notified and now watches 1. Nothing
> > else really happens.
> >
> > But if the nodes are 1, 2, 3 and 1 disappears 2 gets notified
> > and says, in effect "I'm first in line so I will be leader". 3
> > doesn't get any notification.
> >
> > bq. I wonder why the queue is not rotated until the new and preferred
> > leader is at front (position 0)
> >
> > Because then you'd have N leader changes where N is the number
> > of nodes between the preferred leader and the actual leader in
> > the leader election queue at the start. That could result in 100s of
> > changes when only 1 is desired. The original case for this was
> > exactly that, there could be 100s of shards and several tens
> > to 100s of replicas.
> >
> > Hmmmm, I suppose you wouldn't have that many leader changes if
> > you sent the ephemeral node in the _second_ position to the end
> > of the queue until the preferredLeader became the one in the second
> > position. You'd still have a watch fired for every requeueing though. I
> > haven't measured the cost there. That would also be an added
> > burden for the Overseer, which has been overwhelmed in the past.
> >
> > I'm not against that solution, I don't have any real data to
> > evaluate.
> >
> > bq. ....is possible to have more than one electionNode with the
> > same sequence number.
> >
> > Yeah, and it causes complexity, but I don't have a good way around it. This is
> > related to your sorting question. ZK itself has a simple ordering, sequential
> > sequence numbers. Having two the same is the only way I could see (actually I
> > patterned it off some other code) to insert an ephemeral node second. What you
> > have then is two nodes "tied" for second by having the same sequence numbers.
> >
> > Which one ZK thinks is second (and
> > thus which one becomes the leader if the zero'th ephemeral node disappears)
> > is based on sorting which includes the session ID, so there's code in there that
> > has to deal with sending the non-preferred node that's tied for second to the
> > end of the queue. That's the code that got changed during various
> > refactorings that I didn't take part in, and the code that's messed up.
> >
> > bq. Wherever I see any java code to get the content from the queue it
> > is sorted. Where is the sense of this?
> >
> > This is implied by the above, but explicitly so the Java code can see the queue
> > in the same order that ZK does and "do the right thing". In this case
> > assign the preferred leader's ephemeral node with the same sequence number
> > that the current second-in-line has and move the current second-in-line to the
> > end of the queue.
> >
> > All that said, this code was written several years ago and I haven't looked at
> > whether there are better methods available now. The actions that are necessary
> > are:
> >
> > 1> ensure that the preferredLeader is the only node watching the leader in the
> >      leader election queue
> >
> > 2> re-queue the leader at the end of the leader election queue. Since we'd be
> > sure the preferredLeader is watching the leader, this action would elect
> > the proper node as the leader.
> >
> > Hmmm, note to self. It would help folks in the future if I, you know, documented
> > those two points in the code. Siiggghhh.
> >
> > Last night I found what I  _think_ is the problem I was having. Note that the
> > current code has three problems. I think I have fixes for all of them:
> >
> > 1> assigning the preferredLeader (or any SHARDUNIQUE property) does not
> >       properly remove that property from other replicas in the shard if
> >      present. So you may have multiple preferredLeaders in a shard.
> >
> > 2> the code to resolve tied sequence numbers had been changed
> >       during some refactoring so the wrong node could be elected.
> >
> > 3> the response from the rebalanceleaders command isn't very useful, it's
> >       on my plate to fix that. Partly it was not reporting useful
> >       info, and partly your comment from the other day that it returns
> >       without verifying the leadership has actually changed is well taken. At
> >       present, it just changes the election queue and assumes that the
> >       right thing happens. The test code was supposed to point out when
> >       that assumption was incorrect, but you know the story there.
> >
> > Currently, the code is pretty ugly in terms of all the junk I put in trying to
> > track this down, but when I clean it up I'll put up a patch. I added some
> > code to restart some of the jettys in the test (it's now "@Slow:") that
> > catches the restart case. Additionally, I changed the test to force
> > unique properties to be concentrated on a particular node then issue the
> > BALANCESHARDUNIQUE command to make sure that <1> above
> > doesn't happen.
> >
> > Meanwhile, if there's an alternative approach that's simpler I'd be all
> > for it.
> >
> > Best,
> > Erick
> >
> > On Wed, Jan 9, 2019 at 1:32 AM Bernd Fehling
> > <be...@uni-bielefeld.de> wrote:
> >>
> >> Yes, your findings are also very strange.
> >> I wonder if we can discover the "inventor" of all this and ask him
> >> how it should work or better how he originally wanted it to work.
> >>
> >> Comments in the code (RebalanceLeaders.java) state that it is possible
> >> to have more than one electionNode with the same sequence number.
> >> Absolutely strange.
> >>
> >> I wonder why the queue is not rotated until the new and preferred
> >> leader is at front (position 0)?
> >> But why is it a queue anyway?
> >> Wherever I see any java code to get the content from the queue it
> >> is sorted. Where is the sense of this?
> >>
> >> Also, the elctionNodes have another attribute with name "ephemeral".
> >> Where is that for and why is it not tested in TestRebalanceLeaders.java?
> >>
> >> Regards, Bernd
> >>
> >>
> >> Am 09.01.19 um 02:31 schrieb Erick Erickson:
> >>> It's weirder than that. In the current test on master, the
> >>> assumption is that the node recorded as leader in ZK
> >>> is actually the leader, see
> >>> TestRebalanceLeaders.checkZkLeadersAgree(). The theory
> >>> is that the identified leader node in ZK is actually the leader
> >>> after the rebalance command. But you're right, I don't see
> >>> an actual check that the collection's status agrees.
> >>>
> >>> That aside, though, there are several problems I'm uncovering
> >>>
> >>> 1> BALANCESHARDUNIQUE can wind up with multiple
> >>> "preferredLeader" properties defined. Some time between
> >>> the original code and now someone refactored a bunch of
> >>> code and missed removing a unique property if it was
> >>> already assigned and being assigned to another replica
> >>> in the same slice.
> >>>
> >>> 2> to make it much worse, I've rewritten the tests
> >>> extensively and I can beast the rewritten tests 1,000
> >>> times and no failures. If I test manually by just issuing
> >>> the commands, everything works fine. By "testing manually"
> >>> I mean (working with 4 Vms, 10 shards 4 replicas)
> >>>> create the collection
> >>>> issue the BALANCESHARDUNIQUE command
> >>>> issue the REBALANCELEADERS command
> >>>
> >>>
> >>> However, if instead I
> >>>> create the collection
> >>>> issue the BALANCESHARDUNIQUE command
> >>>> shut down 3 of 4 Solr instances so all the leaders
> >>>      are on the same host.
> >>>> restart the 3 instances
> >>>> issue the REBALANCELEADERS command then
> >>>      it doesn't work.
> >>>
> >>> At least that's what I think I'm seeing, but it makes no
> >>> real sense yet.
> >>>
> >>> So I'm first trying to understand why my manual test
> >>> fails so regularly, then I can incorporate that setup
> >>> into the unit test (I'm thinking of just shutting down
> >>> and restarting some of the Jetty instances).
> >>>
> >>> But it's a total mystery to me why restarting Solr instances
> >>> should have any effect. But that's certainly not
> >>> something that happens in the current test so I have
> >>> hopes that tracking that down will lead to understanding
> >>> what the invalid assumption I'm making is and we can
> >>> test for that too.,
> >>>
> >>> On Tue, Jan 8, 2019 at 1:42 AM Bernd Fehling
> >>> <be...@uni-bielefeld.de> wrote:
> >>>>
> >>>> Hi Erick,
> >>>>
> >>>> after some more hours of debugging the rough result is, who ever invented
> >>>> this leader election did not check if an action returns the estimated
> >>>> result. There are only checks for exceptions, true/false, new sequence
> >>>> numbers and so on, but never if a leader election to the preferredleader
> >>>> really took place.
> >>>>
> >>>> If doing a rebalanceleaders to preferredleader I also have to check if:
> >>>> - a rebalance took place
> >>>> - the preferredleader has really become leader (and not anyone else)
> >>>>
> >>>> Currently this is not checked and the call rebalanceleaders to preferredleader
> >>>> is like a shot into the dark with hope of success. And thats why any
> >>>> problems have never been discovered or reported.
> >>>>
> >>>> Bernd
> >>>>
> >>>>
> >>>> Am 21.12.18 um 18:00 schrieb Erick Erickson:
> >>>>> I looked at the test last night and it's...disturbing. It succeeds
> >>>>> 100% of the time. Manual testing seems to fail very often.
> >>>>> Of course it was late and I was a bit cross-eyed, so maybe
> >>>>> I wasn't looking at the manual tests correctly. Or maybe the
> >>>>> test is buggy.
> >>>>>
> >>>>> I beasted the test 100x last night and all of them succeeded.
> >>>>>
> >>>>> This was with all NRT replicas.
> >>>>>
> >>>>> Today I'm going to modify the test into a stand-alone program
> >>>>> to see if it's something in the test environment that causes
> >>>>> it to succeed. I've got to get this to fail as a unit test before I
> >>>>> have confidence in any fixes, and also confidence that things
> >>>>> like this will be caught going forward.
> >>>>>
> >>>>> Erick
> >>>>>
> >>>>> On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
> >>>>> <be...@uni-bielefeld.de> wrote:
> >>>>>>
> >>>>>> As far as I could see with debugger there is still a problem in requeing.
> >>>>>>
> >>>>>> There is a watcher and it is recognized that the watcher is not a preferredleader.
> >>>>>> So it tries to locate a preferredleader with success.
> >>>>>> It then calls makeReplicaFirstWatcher and gets a new sequence number for
> >>>>>> the preferredleader replica. But now we have two replicas with the same
> >>>>>> sequence number. One replica which already owns that sequence number and
> >>>>>> the replica which got the new (and the same) number as new sequence number.
> >>>>>> It now tries to solve this with queueNodesWithSameSequence.
> >>>>>> Might be something in rejoinElection.
> >>>>>> At least the call to rejoinElection seems right. For preferredleader it
> >>>>>> is true for rejoinAtHead and for the other replica with same sequence number
> >>>>>> it is false for rejoinAtHead.
> >>>>>>
> >>>>>> A test case should have 3 shards with 3 cores per shard and should try to
> >>>>>> set preferredleader to different replicas at random. And then try to
> >>>>>> rebalance and check the results.
> >>>>>>
> >>>>>> So far, regards, Bernd
> >>>>>>
> >>>>>>
> >>>>>> Am 21.12.18 um 07:11 schrieb Erick Erickson:
> >>>>>>> I'm reworking the test case, so hold off on doing that. If you want to
> >>>>>>> raise a JIRA, though. please do and attach your patch...
> >>>>>>>
> >>>>>>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
> >>>>>>>>
> >>>>>>>> Nothing that I know of was _intentionally_ changed with this between
> >>>>>>>> 6x and 7x. That said, nothing that I know of was done to verify that
> >>>>>>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
> >>>>>>>> test "TestRebalanceLeaders" for this functionality that has run since
> >>>>>>>> the feature was put in, but it has _not_ been modified to create TLOG
> >>>>>>>> and PULL replicas and test with those.
> >>>>>>>>
> >>>>>>>> For this patch to be complete, we should either extend that test or
> >>>>>>>> make another that fails without this patch and succeeds with it.
> >>>>>>>>
> >>>>>>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
> >>>>>>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
> >>>>>>>> various cases.
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Erick
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
> >>>>>>>> <be...@uni-bielefeld.de> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi Vadim,
> >>>>>>>>> I just tried it with 6.6.5.
> >>>>>>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
> >>>>>>>>> one shard to become leader. But noticed that one shard already was
> >>>>>>>>> leader. No errors or exceptions in logs.
> >>>>>>>>> May be I should enable debug logging and try again to see all logging
> >>>>>>>>> messages from the patch.
> >>>>>>>>>
> >>>>>>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
> >>>>>>>>> it works for you.
> >>>>>>>>>
> >>>>>>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
> >>>>>>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
> >>>>>>>>>
> >>>>>>>>> Regards, Bernd
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
> >>>>>>>>>> Yes! It works!
> >>>>>>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
> >>>>>>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
> >>>>>>>>>> Original patch was slightly incompatible with 7.6.0
> >>>>>>>>>> I hope this patch will help to try this feature with 7.6
> >>>>>>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
> >>>>>>>>>>
> >>>>>>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
> >>>>>>>>>> But new replica types made it very helpful to keep big clusters in order...
> >>>>>>>>>>
> >>>>>>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
> >>>>>>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
> >>>>>>>>>>
>

Re: REBALANCELEADERS is not reliable

Posted by Bernd Fehling <be...@uni-bielefeld.de>.
Hi Erik,

that is very valuable info I missed.
Shouldn't that belong into an issue about rework at REBALANCELEADERS?

With your explanation the use of a queue makes sense and now I see some of
the logic behind.
- there is the leader and the firstWatcher
- if firstWatcher goes down or is inactive that one should positioned
   at end of queue which automatically gives the baton to the next in queue
- preferredLeader has precedence if state=active
- there can be only one preferredLeader per shard

Any  more rules?

 From debugging I know there are a lot of actions taken behind the scene
and many things can go wrong. Parts included are ZkStateReader, Overseer,
ShardLeaderElectionContextBase, ShardLeaderElectionContext, ElectionContext,
LeaderElector, some Requests and a HttpSolrCall to REJOINLEADERELECTION.

An info I see when rebalance goes wrong is from ShardLeaderElectionContextBase:
"No version found for ephemeral leader parent node, won't remove previous leader registration."

Regards, Bernd


Am 09.01.19 um 17:22 schrieb Erick Erickson:
> Executive summary:
> 
> The central problem is "how can I insert an ephemeral node
> in a specific place in a ZK queue". The code could be much,
> much simpler if there were a reliable way to do just that. I haven't
> looked at more recent ZKs to see if it's possible, I'd love it if
> there were a better way.
> 
> On to details:
> 
> bq.  wonder if we can discover the "inventor" of all this and ask him
> how it should work
> 
> Yeah, I can contact that clown. That would be me ;)
> 
> The way leader election works is a  ZK recipe where each
> ephemeral node only watches the one in front of it. When a
> node is deleted, the one watching it is notified.
> 
> So let's say we have nodes in this order: 1 watched by 2
> watched by 3... In this case 1 is the leader. If 2 should
> disappear, 3 gets notified and now watches 1. Nothing
> else really happens.
> 
> But if the nodes are 1, 2, 3 and 1 disappears 2 gets notified
> and says, in effect "I'm first in line so I will be leader". 3
> doesn't get any notification.
> 
> bq. I wonder why the queue is not rotated until the new and preferred
> leader is at front (position 0)
> 
> Because then you'd have N leader changes where N is the number
> of nodes between the preferred leader and the actual leader in
> the leader election queue at the start. That could result in 100s of
> changes when only 1 is desired. The original case for this was
> exactly that, there could be 100s of shards and several tens
> to 100s of replicas.
> 
> Hmmmm, I suppose you wouldn't have that many leader changes if
> you sent the ephemeral node in the _second_ position to the end
> of the queue until the preferredLeader became the one in the second
> position. You'd still have a watch fired for every requeueing though. I
> haven't measured the cost there. That would also be an added
> burden for the Overseer, which has been overwhelmed in the past.
> 
> I'm not against that solution, I don't have any real data to
> evaluate.
> 
> bq. ....is possible to have more than one electionNode with the
> same sequence number.
> 
> Yeah, and it causes complexity, but I don't have a good way around it. This is
> related to your sorting question. ZK itself has a simple ordering, sequential
> sequence numbers. Having two the same is the only way I could see (actually I
> patterned it off some other code) to insert an ephemeral node second. What you
> have then is two nodes "tied" for second by having the same sequence numbers.
> 
> Which one ZK thinks is second (and
> thus which one becomes the leader if the zero'th ephemeral node disappears)
> is based on sorting which includes the session ID, so there's code in there that
> has to deal with sending the non-preferred node that's tied for second to the
> end of the queue. That's the code that got changed during various
> refactorings that I didn't take part in, and the code that's messed up.
> 
> bq. Wherever I see any java code to get the content from the queue it
> is sorted. Where is the sense of this?
> 
> This is implied by the above, but explicitly so the Java code can see the queue
> in the same order that ZK does and "do the right thing". In this case
> assign the preferred leader's ephemeral node with the same sequence number
> that the current second-in-line has and move the current second-in-line to the
> end of the queue.
> 
> All that said, this code was written several years ago and I haven't looked at
> whether there are better methods available now. The actions that are necessary
> are:
> 
> 1> ensure that the preferredLeader is the only node watching the leader in the
>      leader election queue
> 
> 2> re-queue the leader at the end of the leader election queue. Since we'd be
> sure the preferredLeader is watching the leader, this action would elect
> the proper node as the leader.
> 
> Hmmm, note to self. It would help folks in the future if I, you know, documented
> those two points in the code. Siiggghhh.
> 
> Last night I found what I  _think_ is the problem I was having. Note that the
> current code has three problems. I think I have fixes for all of them:
> 
> 1> assigning the preferredLeader (or any SHARDUNIQUE property) does not
>       properly remove that property from other replicas in the shard if
>      present. So you may have multiple preferredLeaders in a shard.
> 
> 2> the code to resolve tied sequence numbers had been changed
>       during some refactoring so the wrong node could be elected.
> 
> 3> the response from the rebalanceleaders command isn't very useful, it's
>       on my plate to fix that. Partly it was not reporting useful
>       info, and partly your comment from the other day that it returns
>       without verifying the leadership has actually changed is well taken. At
>       present, it just changes the election queue and assumes that the
>       right thing happens. The test code was supposed to point out when
>       that assumption was incorrect, but you know the story there.
> 
> Currently, the code is pretty ugly in terms of all the junk I put in trying to
> track this down, but when I clean it up I'll put up a patch. I added some
> code to restart some of the jettys in the test (it's now "@Slow:") that
> catches the restart case. Additionally, I changed the test to force
> unique properties to be concentrated on a particular node then issue the
> BALANCESHARDUNIQUE command to make sure that <1> above
> doesn't happen.
> 
> Meanwhile, if there's an alternative approach that's simpler I'd be all
> for it.
> 
> Best,
> Erick
> 
> On Wed, Jan 9, 2019 at 1:32 AM Bernd Fehling
> <be...@uni-bielefeld.de> wrote:
>>
>> Yes, your findings are also very strange.
>> I wonder if we can discover the "inventor" of all this and ask him
>> how it should work or better how he originally wanted it to work.
>>
>> Comments in the code (RebalanceLeaders.java) state that it is possible
>> to have more than one electionNode with the same sequence number.
>> Absolutely strange.
>>
>> I wonder why the queue is not rotated until the new and preferred
>> leader is at front (position 0)?
>> But why is it a queue anyway?
>> Wherever I see any java code to get the content from the queue it
>> is sorted. Where is the sense of this?
>>
>> Also, the elctionNodes have another attribute with name "ephemeral".
>> Where is that for and why is it not tested in TestRebalanceLeaders.java?
>>
>> Regards, Bernd
>>
>>
>> Am 09.01.19 um 02:31 schrieb Erick Erickson:
>>> It's weirder than that. In the current test on master, the
>>> assumption is that the node recorded as leader in ZK
>>> is actually the leader, see
>>> TestRebalanceLeaders.checkZkLeadersAgree(). The theory
>>> is that the identified leader node in ZK is actually the leader
>>> after the rebalance command. But you're right, I don't see
>>> an actual check that the collection's status agrees.
>>>
>>> That aside, though, there are several problems I'm uncovering
>>>
>>> 1> BALANCESHARDUNIQUE can wind up with multiple
>>> "preferredLeader" properties defined. Some time between
>>> the original code and now someone refactored a bunch of
>>> code and missed removing a unique property if it was
>>> already assigned and being assigned to another replica
>>> in the same slice.
>>>
>>> 2> to make it much worse, I've rewritten the tests
>>> extensively and I can beast the rewritten tests 1,000
>>> times and no failures. If I test manually by just issuing
>>> the commands, everything works fine. By "testing manually"
>>> I mean (working with 4 Vms, 10 shards 4 replicas)
>>>> create the collection
>>>> issue the BALANCESHARDUNIQUE command
>>>> issue the REBALANCELEADERS command
>>>
>>>
>>> However, if instead I
>>>> create the collection
>>>> issue the BALANCESHARDUNIQUE command
>>>> shut down 3 of 4 Solr instances so all the leaders
>>>      are on the same host.
>>>> restart the 3 instances
>>>> issue the REBALANCELEADERS command then
>>>      it doesn't work.
>>>
>>> At least that's what I think I'm seeing, but it makes no
>>> real sense yet.
>>>
>>> So I'm first trying to understand why my manual test
>>> fails so regularly, then I can incorporate that setup
>>> into the unit test (I'm thinking of just shutting down
>>> and restarting some of the Jetty instances).
>>>
>>> But it's a total mystery to me why restarting Solr instances
>>> should have any effect. But that's certainly not
>>> something that happens in the current test so I have
>>> hopes that tracking that down will lead to understanding
>>> what the invalid assumption I'm making is and we can
>>> test for that too.,
>>>
>>> On Tue, Jan 8, 2019 at 1:42 AM Bernd Fehling
>>> <be...@uni-bielefeld.de> wrote:
>>>>
>>>> Hi Erick,
>>>>
>>>> after some more hours of debugging the rough result is, who ever invented
>>>> this leader election did not check if an action returns the estimated
>>>> result. There are only checks for exceptions, true/false, new sequence
>>>> numbers and so on, but never if a leader election to the preferredleader
>>>> really took place.
>>>>
>>>> If doing a rebalanceleaders to preferredleader I also have to check if:
>>>> - a rebalance took place
>>>> - the preferredleader has really become leader (and not anyone else)
>>>>
>>>> Currently this is not checked and the call rebalanceleaders to preferredleader
>>>> is like a shot into the dark with hope of success. And thats why any
>>>> problems have never been discovered or reported.
>>>>
>>>> Bernd
>>>>
>>>>
>>>> Am 21.12.18 um 18:00 schrieb Erick Erickson:
>>>>> I looked at the test last night and it's...disturbing. It succeeds
>>>>> 100% of the time. Manual testing seems to fail very often.
>>>>> Of course it was late and I was a bit cross-eyed, so maybe
>>>>> I wasn't looking at the manual tests correctly. Or maybe the
>>>>> test is buggy.
>>>>>
>>>>> I beasted the test 100x last night and all of them succeeded.
>>>>>
>>>>> This was with all NRT replicas.
>>>>>
>>>>> Today I'm going to modify the test into a stand-alone program
>>>>> to see if it's something in the test environment that causes
>>>>> it to succeed. I've got to get this to fail as a unit test before I
>>>>> have confidence in any fixes, and also confidence that things
>>>>> like this will be caught going forward.
>>>>>
>>>>> Erick
>>>>>
>>>>> On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
>>>>> <be...@uni-bielefeld.de> wrote:
>>>>>>
>>>>>> As far as I could see with debugger there is still a problem in requeing.
>>>>>>
>>>>>> There is a watcher and it is recognized that the watcher is not a preferredleader.
>>>>>> So it tries to locate a preferredleader with success.
>>>>>> It then calls makeReplicaFirstWatcher and gets a new sequence number for
>>>>>> the preferredleader replica. But now we have two replicas with the same
>>>>>> sequence number. One replica which already owns that sequence number and
>>>>>> the replica which got the new (and the same) number as new sequence number.
>>>>>> It now tries to solve this with queueNodesWithSameSequence.
>>>>>> Might be something in rejoinElection.
>>>>>> At least the call to rejoinElection seems right. For preferredleader it
>>>>>> is true for rejoinAtHead and for the other replica with same sequence number
>>>>>> it is false for rejoinAtHead.
>>>>>>
>>>>>> A test case should have 3 shards with 3 cores per shard and should try to
>>>>>> set preferredleader to different replicas at random. And then try to
>>>>>> rebalance and check the results.
>>>>>>
>>>>>> So far, regards, Bernd
>>>>>>
>>>>>>
>>>>>> Am 21.12.18 um 07:11 schrieb Erick Erickson:
>>>>>>> I'm reworking the test case, so hold off on doing that. If you want to
>>>>>>> raise a JIRA, though. please do and attach your patch...
>>>>>>>
>>>>>>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
>>>>>>>>
>>>>>>>> Nothing that I know of was _intentionally_ changed with this between
>>>>>>>> 6x and 7x. That said, nothing that I know of was done to verify that
>>>>>>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
>>>>>>>> test "TestRebalanceLeaders" for this functionality that has run since
>>>>>>>> the feature was put in, but it has _not_ been modified to create TLOG
>>>>>>>> and PULL replicas and test with those.
>>>>>>>>
>>>>>>>> For this patch to be complete, we should either extend that test or
>>>>>>>> make another that fails without this patch and succeeds with it.
>>>>>>>>
>>>>>>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
>>>>>>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
>>>>>>>> various cases.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Erick
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
>>>>>>>> <be...@uni-bielefeld.de> wrote:
>>>>>>>>>
>>>>>>>>> Hi Vadim,
>>>>>>>>> I just tried it with 6.6.5.
>>>>>>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
>>>>>>>>> one shard to become leader. But noticed that one shard already was
>>>>>>>>> leader. No errors or exceptions in logs.
>>>>>>>>> May be I should enable debug logging and try again to see all logging
>>>>>>>>> messages from the patch.
>>>>>>>>>
>>>>>>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
>>>>>>>>> it works for you.
>>>>>>>>>
>>>>>>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
>>>>>>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
>>>>>>>>>
>>>>>>>>> Regards, Bernd
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
>>>>>>>>>> Yes! It works!
>>>>>>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
>>>>>>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
>>>>>>>>>> Original patch was slightly incompatible with 7.6.0
>>>>>>>>>> I hope this patch will help to try this feature with 7.6
>>>>>>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
>>>>>>>>>>
>>>>>>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
>>>>>>>>>> But new replica types made it very helpful to keep big clusters in order...
>>>>>>>>>>
>>>>>>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
>>>>>>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
>>>>>>>>>>


Re: REBALANCELEADERS is not reliable

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

The central problem is "how can I insert an ephemeral node
in a specific place in a ZK queue". The code could be much,
much simpler if there were a reliable way to do just that. I haven't
looked at more recent ZKs to see if it's possible, I'd love it if
there were a better way.

On to details:

bq.  wonder if we can discover the "inventor" of all this and ask him
how it should work

Yeah, I can contact that clown. That would be me ;)

The way leader election works is a  ZK recipe where each
ephemeral node only watches the one in front of it. When a
node is deleted, the one watching it is notified.

So let's say we have nodes in this order: 1 watched by 2
watched by 3... In this case 1 is the leader. If 2 should
disappear, 3 gets notified and now watches 1. Nothing
else really happens.

But if the nodes are 1, 2, 3 and 1 disappears 2 gets notified
and says, in effect "I'm first in line so I will be leader". 3
doesn't get any notification.

bq. I wonder why the queue is not rotated until the new and preferred
leader is at front (position 0)

Because then you'd have N leader changes where N is the number
of nodes between the preferred leader and the actual leader in
the leader election queue at the start. That could result in 100s of
changes when only 1 is desired. The original case for this was
exactly that, there could be 100s of shards and several tens
to 100s of replicas.

Hmmmm, I suppose you wouldn't have that many leader changes if
you sent the ephemeral node in the _second_ position to the end
of the queue until the preferredLeader became the one in the second
position. You'd still have a watch fired for every requeueing though. I
haven't measured the cost there. That would also be an added
burden for the Overseer, which has been overwhelmed in the past.

I'm not against that solution, I don't have any real data to
evaluate.

bq. ....is possible to have more than one electionNode with the
same sequence number.

Yeah, and it causes complexity, but I don't have a good way around it. This is
related to your sorting question. ZK itself has a simple ordering, sequential
sequence numbers. Having two the same is the only way I could see (actually I
patterned it off some other code) to insert an ephemeral node second. What you
have then is two nodes "tied" for second by having the same sequence numbers.

Which one ZK thinks is second (and
thus which one becomes the leader if the zero'th ephemeral node disappears)
is based on sorting which includes the session ID, so there's code in there that
has to deal with sending the non-preferred node that's tied for second to the
end of the queue. That's the code that got changed during various
refactorings that I didn't take part in, and the code that's messed up.

bq. Wherever I see any java code to get the content from the queue it
is sorted. Where is the sense of this?

This is implied by the above, but explicitly so the Java code can see the queue
in the same order that ZK does and "do the right thing". In this case
assign the preferred leader's ephemeral node with the same sequence number
that the current second-in-line has and move the current second-in-line to the
end of the queue.

All that said, this code was written several years ago and I haven't looked at
whether there are better methods available now. The actions that are necessary
are:

1> ensure that the preferredLeader is the only node watching the leader in the
    leader election queue

2> re-queue the leader at the end of the leader election queue. Since we'd be
sure the preferredLeader is watching the leader, this action would elect
the proper node as the leader.

Hmmm, note to self. It would help folks in the future if I, you know, documented
those two points in the code. Siiggghhh.

Last night I found what I  _think_ is the problem I was having. Note that the
current code has three problems. I think I have fixes for all of them:

1> assigning the preferredLeader (or any SHARDUNIQUE property) does not
     properly remove that property from other replicas in the shard if
    present. So you may have multiple preferredLeaders in a shard.

2> the code to resolve tied sequence numbers had been changed
     during some refactoring so the wrong node could be elected.

3> the response from the rebalanceleaders command isn't very useful, it's
     on my plate to fix that. Partly it was not reporting useful
     info, and partly your comment from the other day that it returns
     without verifying the leadership has actually changed is well taken. At
     present, it just changes the election queue and assumes that the
     right thing happens. The test code was supposed to point out when
     that assumption was incorrect, but you know the story there.

Currently, the code is pretty ugly in terms of all the junk I put in trying to
track this down, but when I clean it up I'll put up a patch. I added some
code to restart some of the jettys in the test (it's now "@Slow:") that
catches the restart case. Additionally, I changed the test to force
unique properties to be concentrated on a particular node then issue the
BALANCESHARDUNIQUE command to make sure that <1> above
doesn't happen.

Meanwhile, if there's an alternative approach that's simpler I'd be all
for it.

Best,
Erick

On Wed, Jan 9, 2019 at 1:32 AM Bernd Fehling
<be...@uni-bielefeld.de> wrote:
>
> Yes, your findings are also very strange.
> I wonder if we can discover the "inventor" of all this and ask him
> how it should work or better how he originally wanted it to work.
>
> Comments in the code (RebalanceLeaders.java) state that it is possible
> to have more than one electionNode with the same sequence number.
> Absolutely strange.
>
> I wonder why the queue is not rotated until the new and preferred
> leader is at front (position 0)?
> But why is it a queue anyway?
> Wherever I see any java code to get the content from the queue it
> is sorted. Where is the sense of this?
>
> Also, the elctionNodes have another attribute with name "ephemeral".
> Where is that for and why is it not tested in TestRebalanceLeaders.java?
>
> Regards, Bernd
>
>
> Am 09.01.19 um 02:31 schrieb Erick Erickson:
> > It's weirder than that. In the current test on master, the
> > assumption is that the node recorded as leader in ZK
> > is actually the leader, see
> > TestRebalanceLeaders.checkZkLeadersAgree(). The theory
> > is that the identified leader node in ZK is actually the leader
> > after the rebalance command. But you're right, I don't see
> > an actual check that the collection's status agrees.
> >
> > That aside, though, there are several problems I'm uncovering
> >
> > 1> BALANCESHARDUNIQUE can wind up with multiple
> > "preferredLeader" properties defined. Some time between
> > the original code and now someone refactored a bunch of
> > code and missed removing a unique property if it was
> > already assigned and being assigned to another replica
> > in the same slice.
> >
> > 2> to make it much worse, I've rewritten the tests
> > extensively and I can beast the rewritten tests 1,000
> > times and no failures. If I test manually by just issuing
> > the commands, everything works fine. By "testing manually"
> > I mean (working with 4 Vms, 10 shards 4 replicas)
> >> create the collection
> >> issue the BALANCESHARDUNIQUE command
> >> issue the REBALANCELEADERS command
> >
> >
> > However, if instead I
> >> create the collection
> >> issue the BALANCESHARDUNIQUE command
> >> shut down 3 of 4 Solr instances so all the leaders
> >     are on the same host.
> >> restart the 3 instances
> >> issue the REBALANCELEADERS command then
> >     it doesn't work.
> >
> > At least that's what I think I'm seeing, but it makes no
> > real sense yet.
> >
> > So I'm first trying to understand why my manual test
> > fails so regularly, then I can incorporate that setup
> > into the unit test (I'm thinking of just shutting down
> > and restarting some of the Jetty instances).
> >
> > But it's a total mystery to me why restarting Solr instances
> > should have any effect. But that's certainly not
> > something that happens in the current test so I have
> > hopes that tracking that down will lead to understanding
> > what the invalid assumption I'm making is and we can
> > test for that too.,
> >
> > On Tue, Jan 8, 2019 at 1:42 AM Bernd Fehling
> > <be...@uni-bielefeld.de> wrote:
> >>
> >> Hi Erick,
> >>
> >> after some more hours of debugging the rough result is, who ever invented
> >> this leader election did not check if an action returns the estimated
> >> result. There are only checks for exceptions, true/false, new sequence
> >> numbers and so on, but never if a leader election to the preferredleader
> >> really took place.
> >>
> >> If doing a rebalanceleaders to preferredleader I also have to check if:
> >> - a rebalance took place
> >> - the preferredleader has really become leader (and not anyone else)
> >>
> >> Currently this is not checked and the call rebalanceleaders to preferredleader
> >> is like a shot into the dark with hope of success. And thats why any
> >> problems have never been discovered or reported.
> >>
> >> Bernd
> >>
> >>
> >> Am 21.12.18 um 18:00 schrieb Erick Erickson:
> >>> I looked at the test last night and it's...disturbing. It succeeds
> >>> 100% of the time. Manual testing seems to fail very often.
> >>> Of course it was late and I was a bit cross-eyed, so maybe
> >>> I wasn't looking at the manual tests correctly. Or maybe the
> >>> test is buggy.
> >>>
> >>> I beasted the test 100x last night and all of them succeeded.
> >>>
> >>> This was with all NRT replicas.
> >>>
> >>> Today I'm going to modify the test into a stand-alone program
> >>> to see if it's something in the test environment that causes
> >>> it to succeed. I've got to get this to fail as a unit test before I
> >>> have confidence in any fixes, and also confidence that things
> >>> like this will be caught going forward.
> >>>
> >>> Erick
> >>>
> >>> On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
> >>> <be...@uni-bielefeld.de> wrote:
> >>>>
> >>>> As far as I could see with debugger there is still a problem in requeing.
> >>>>
> >>>> There is a watcher and it is recognized that the watcher is not a preferredleader.
> >>>> So it tries to locate a preferredleader with success.
> >>>> It then calls makeReplicaFirstWatcher and gets a new sequence number for
> >>>> the preferredleader replica. But now we have two replicas with the same
> >>>> sequence number. One replica which already owns that sequence number and
> >>>> the replica which got the new (and the same) number as new sequence number.
> >>>> It now tries to solve this with queueNodesWithSameSequence.
> >>>> Might be something in rejoinElection.
> >>>> At least the call to rejoinElection seems right. For preferredleader it
> >>>> is true for rejoinAtHead and for the other replica with same sequence number
> >>>> it is false for rejoinAtHead.
> >>>>
> >>>> A test case should have 3 shards with 3 cores per shard and should try to
> >>>> set preferredleader to different replicas at random. And then try to
> >>>> rebalance and check the results.
> >>>>
> >>>> So far, regards, Bernd
> >>>>
> >>>>
> >>>> Am 21.12.18 um 07:11 schrieb Erick Erickson:
> >>>>> I'm reworking the test case, so hold off on doing that. If you want to
> >>>>> raise a JIRA, though. please do and attach your patch...
> >>>>>
> >>>>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
> >>>>>>
> >>>>>> Nothing that I know of was _intentionally_ changed with this between
> >>>>>> 6x and 7x. That said, nothing that I know of was done to verify that
> >>>>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
> >>>>>> test "TestRebalanceLeaders" for this functionality that has run since
> >>>>>> the feature was put in, but it has _not_ been modified to create TLOG
> >>>>>> and PULL replicas and test with those.
> >>>>>>
> >>>>>> For this patch to be complete, we should either extend that test or
> >>>>>> make another that fails without this patch and succeeds with it.
> >>>>>>
> >>>>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
> >>>>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
> >>>>>> various cases.
> >>>>>>
> >>>>>> Best,
> >>>>>> Erick
> >>>>>>
> >>>>>>
> >>>>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
> >>>>>> <be...@uni-bielefeld.de> wrote:
> >>>>>>>
> >>>>>>> Hi Vadim,
> >>>>>>> I just tried it with 6.6.5.
> >>>>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
> >>>>>>> one shard to become leader. But noticed that one shard already was
> >>>>>>> leader. No errors or exceptions in logs.
> >>>>>>> May be I should enable debug logging and try again to see all logging
> >>>>>>> messages from the patch.
> >>>>>>>
> >>>>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
> >>>>>>> it works for you.
> >>>>>>>
> >>>>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
> >>>>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
> >>>>>>>
> >>>>>>> Regards, Bernd
> >>>>>>>
> >>>>>>>
> >>>>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
> >>>>>>>> Yes! It works!
> >>>>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
> >>>>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
> >>>>>>>> Original patch was slightly incompatible with 7.6.0
> >>>>>>>> I hope this patch will help to try this feature with 7.6
> >>>>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
> >>>>>>>>
> >>>>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
> >>>>>>>> But new replica types made it very helpful to keep big clusters in order...
> >>>>>>>>
> >>>>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
> >>>>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
> >>>>>>>>

Re: REBALANCELEADERS is not reliable

Posted by Bernd Fehling <be...@uni-bielefeld.de>.
Yes, your findings are also very strange.
I wonder if we can discover the "inventor" of all this and ask him
how it should work or better how he originally wanted it to work.

Comments in the code (RebalanceLeaders.java) state that it is possible
to have more than one electionNode with the same sequence number.
Absolutely strange.

I wonder why the queue is not rotated until the new and preferred
leader is at front (position 0)?
But why is it a queue anyway?
Wherever I see any java code to get the content from the queue it
is sorted. Where is the sense of this?

Also, the elctionNodes have another attribute with name "ephemeral".
Where is that for and why is it not tested in TestRebalanceLeaders.java?

Regards, Bernd


Am 09.01.19 um 02:31 schrieb Erick Erickson:
> It's weirder than that. In the current test on master, the
> assumption is that the node recorded as leader in ZK
> is actually the leader, see
> TestRebalanceLeaders.checkZkLeadersAgree(). The theory
> is that the identified leader node in ZK is actually the leader
> after the rebalance command. But you're right, I don't see
> an actual check that the collection's status agrees.
> 
> That aside, though, there are several problems I'm uncovering
> 
> 1> BALANCESHARDUNIQUE can wind up with multiple
> "preferredLeader" properties defined. Some time between
> the original code and now someone refactored a bunch of
> code and missed removing a unique property if it was
> already assigned and being assigned to another replica
> in the same slice.
> 
> 2> to make it much worse, I've rewritten the tests
> extensively and I can beast the rewritten tests 1,000
> times and no failures. If I test manually by just issuing
> the commands, everything works fine. By "testing manually"
> I mean (working with 4 Vms, 10 shards 4 replicas)
>> create the collection
>> issue the BALANCESHARDUNIQUE command
>> issue the REBALANCELEADERS command
> 
> 
> However, if instead I
>> create the collection
>> issue the BALANCESHARDUNIQUE command
>> shut down 3 of 4 Solr instances so all the leaders
>     are on the same host.
>> restart the 3 instances
>> issue the REBALANCELEADERS command then
>     it doesn't work.
> 
> At least that's what I think I'm seeing, but it makes no
> real sense yet.
> 
> So I'm first trying to understand why my manual test
> fails so regularly, then I can incorporate that setup
> into the unit test (I'm thinking of just shutting down
> and restarting some of the Jetty instances).
> 
> But it's a total mystery to me why restarting Solr instances
> should have any effect. But that's certainly not
> something that happens in the current test so I have
> hopes that tracking that down will lead to understanding
> what the invalid assumption I'm making is and we can
> test for that too.,
> 
> On Tue, Jan 8, 2019 at 1:42 AM Bernd Fehling
> <be...@uni-bielefeld.de> wrote:
>>
>> Hi Erick,
>>
>> after some more hours of debugging the rough result is, who ever invented
>> this leader election did not check if an action returns the estimated
>> result. There are only checks for exceptions, true/false, new sequence
>> numbers and so on, but never if a leader election to the preferredleader
>> really took place.
>>
>> If doing a rebalanceleaders to preferredleader I also have to check if:
>> - a rebalance took place
>> - the preferredleader has really become leader (and not anyone else)
>>
>> Currently this is not checked and the call rebalanceleaders to preferredleader
>> is like a shot into the dark with hope of success. And thats why any
>> problems have never been discovered or reported.
>>
>> Bernd
>>
>>
>> Am 21.12.18 um 18:00 schrieb Erick Erickson:
>>> I looked at the test last night and it's...disturbing. It succeeds
>>> 100% of the time. Manual testing seems to fail very often.
>>> Of course it was late and I was a bit cross-eyed, so maybe
>>> I wasn't looking at the manual tests correctly. Or maybe the
>>> test is buggy.
>>>
>>> I beasted the test 100x last night and all of them succeeded.
>>>
>>> This was with all NRT replicas.
>>>
>>> Today I'm going to modify the test into a stand-alone program
>>> to see if it's something in the test environment that causes
>>> it to succeed. I've got to get this to fail as a unit test before I
>>> have confidence in any fixes, and also confidence that things
>>> like this will be caught going forward.
>>>
>>> Erick
>>>
>>> On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
>>> <be...@uni-bielefeld.de> wrote:
>>>>
>>>> As far as I could see with debugger there is still a problem in requeing.
>>>>
>>>> There is a watcher and it is recognized that the watcher is not a preferredleader.
>>>> So it tries to locate a preferredleader with success.
>>>> It then calls makeReplicaFirstWatcher and gets a new sequence number for
>>>> the preferredleader replica. But now we have two replicas with the same
>>>> sequence number. One replica which already owns that sequence number and
>>>> the replica which got the new (and the same) number as new sequence number.
>>>> It now tries to solve this with queueNodesWithSameSequence.
>>>> Might be something in rejoinElection.
>>>> At least the call to rejoinElection seems right. For preferredleader it
>>>> is true for rejoinAtHead and for the other replica with same sequence number
>>>> it is false for rejoinAtHead.
>>>>
>>>> A test case should have 3 shards with 3 cores per shard and should try to
>>>> set preferredleader to different replicas at random. And then try to
>>>> rebalance and check the results.
>>>>
>>>> So far, regards, Bernd
>>>>
>>>>
>>>> Am 21.12.18 um 07:11 schrieb Erick Erickson:
>>>>> I'm reworking the test case, so hold off on doing that. If you want to
>>>>> raise a JIRA, though. please do and attach your patch...
>>>>>
>>>>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
>>>>>>
>>>>>> Nothing that I know of was _intentionally_ changed with this between
>>>>>> 6x and 7x. That said, nothing that I know of was done to verify that
>>>>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
>>>>>> test "TestRebalanceLeaders" for this functionality that has run since
>>>>>> the feature was put in, but it has _not_ been modified to create TLOG
>>>>>> and PULL replicas and test with those.
>>>>>>
>>>>>> For this patch to be complete, we should either extend that test or
>>>>>> make another that fails without this patch and succeeds with it.
>>>>>>
>>>>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
>>>>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
>>>>>> various cases.
>>>>>>
>>>>>> Best,
>>>>>> Erick
>>>>>>
>>>>>>
>>>>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
>>>>>> <be...@uni-bielefeld.de> wrote:
>>>>>>>
>>>>>>> Hi Vadim,
>>>>>>> I just tried it with 6.6.5.
>>>>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
>>>>>>> one shard to become leader. But noticed that one shard already was
>>>>>>> leader. No errors or exceptions in logs.
>>>>>>> May be I should enable debug logging and try again to see all logging
>>>>>>> messages from the patch.
>>>>>>>
>>>>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
>>>>>>> it works for you.
>>>>>>>
>>>>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
>>>>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
>>>>>>>
>>>>>>> Regards, Bernd
>>>>>>>
>>>>>>>
>>>>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
>>>>>>>> Yes! It works!
>>>>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
>>>>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
>>>>>>>> Original patch was slightly incompatible with 7.6.0
>>>>>>>> I hope this patch will help to try this feature with 7.6
>>>>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
>>>>>>>>
>>>>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
>>>>>>>> But new replica types made it very helpful to keep big clusters in order...
>>>>>>>>
>>>>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
>>>>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
>>>>>>>>

Re: REBALANCELEADERS is not reliable

Posted by Erick Erickson <er...@gmail.com>.
It's weirder than that. In the current test on master, the
assumption is that the node recorded as leader in ZK
is actually the leader, see
TestRebalanceLeaders.checkZkLeadersAgree(). The theory
is that the identified leader node in ZK is actually the leader
after the rebalance command. But you're right, I don't see
an actual check that the collection's status agrees.

That aside, though, there are several problems I'm uncovering

1> BALANCESHARDUNIQUE can wind up with multiple
"preferredLeader" properties defined. Some time between
the original code and now someone refactored a bunch of
code and missed removing a unique property if it was
already assigned and being assigned to another replica
in the same slice.

2> to make it much worse, I've rewritten the tests
extensively and I can beast the rewritten tests 1,000
times and no failures. If I test manually by just issuing
the commands, everything works fine. By "testing manually"
I mean (working with 4 Vms, 10 shards 4 replicas)
> create the collection
> issue the BALANCESHARDUNIQUE command
> issue the REBALANCELEADERS command


However, if instead I
> create the collection
> issue the BALANCESHARDUNIQUE command
> shut down 3 of 4 Solr instances so all the leaders
   are on the same host.
> restart the 3 instances
> issue the REBALANCELEADERS command then
   it doesn't work.

At least that's what I think I'm seeing, but it makes no
real sense yet.

So I'm first trying to understand why my manual test
fails so regularly, then I can incorporate that setup
into the unit test (I'm thinking of just shutting down
and restarting some of the Jetty instances).

But it's a total mystery to me why restarting Solr instances
should have any effect. But that's certainly not
something that happens in the current test so I have
hopes that tracking that down will lead to understanding
what the invalid assumption I'm making is and we can
test for that too.,

On Tue, Jan 8, 2019 at 1:42 AM Bernd Fehling
<be...@uni-bielefeld.de> wrote:
>
> Hi Erick,
>
> after some more hours of debugging the rough result is, who ever invented
> this leader election did not check if an action returns the estimated
> result. There are only checks for exceptions, true/false, new sequence
> numbers and so on, but never if a leader election to the preferredleader
> really took place.
>
> If doing a rebalanceleaders to preferredleader I also have to check if:
> - a rebalance took place
> - the preferredleader has really become leader (and not anyone else)
>
> Currently this is not checked and the call rebalanceleaders to preferredleader
> is like a shot into the dark with hope of success. And thats why any
> problems have never been discovered or reported.
>
> Bernd
>
>
> Am 21.12.18 um 18:00 schrieb Erick Erickson:
> > I looked at the test last night and it's...disturbing. It succeeds
> > 100% of the time. Manual testing seems to fail very often.
> > Of course it was late and I was a bit cross-eyed, so maybe
> > I wasn't looking at the manual tests correctly. Or maybe the
> > test is buggy.
> >
> > I beasted the test 100x last night and all of them succeeded.
> >
> > This was with all NRT replicas.
> >
> > Today I'm going to modify the test into a stand-alone program
> > to see if it's something in the test environment that causes
> > it to succeed. I've got to get this to fail as a unit test before I
> > have confidence in any fixes, and also confidence that things
> > like this will be caught going forward.
> >
> > Erick
> >
> > On Fri, Dec 21, 2018 at 3:59 AM Bernd Fehling
> > <be...@uni-bielefeld.de> wrote:
> >>
> >> As far as I could see with debugger there is still a problem in requeing.
> >>
> >> There is a watcher and it is recognized that the watcher is not a preferredleader.
> >> So it tries to locate a preferredleader with success.
> >> It then calls makeReplicaFirstWatcher and gets a new sequence number for
> >> the preferredleader replica. But now we have two replicas with the same
> >> sequence number. One replica which already owns that sequence number and
> >> the replica which got the new (and the same) number as new sequence number.
> >> It now tries to solve this with queueNodesWithSameSequence.
> >> Might be something in rejoinElection.
> >> At least the call to rejoinElection seems right. For preferredleader it
> >> is true for rejoinAtHead and for the other replica with same sequence number
> >> it is false for rejoinAtHead.
> >>
> >> A test case should have 3 shards with 3 cores per shard and should try to
> >> set preferredleader to different replicas at random. And then try to
> >> rebalance and check the results.
> >>
> >> So far, regards, Bernd
> >>
> >>
> >> Am 21.12.18 um 07:11 schrieb Erick Erickson:
> >>> I'm reworking the test case, so hold off on doing that. If you want to
> >>> raise a JIRA, though. please do and attach your patch...
> >>>
> >>> On Thu, Dec 20, 2018 at 10:53 AM Erick Erickson <er...@gmail.com> wrote:
> >>>>
> >>>> Nothing that I know of was _intentionally_ changed with this between
> >>>> 6x and 7x. That said, nothing that I know of was done to verify that
> >>>> TLOG and PULL replicas (added in 7x) were handled correctly. There's a
> >>>> test "TestRebalanceLeaders" for this functionality that has run since
> >>>> the feature was put in, but it has _not_ been modified to create TLOG
> >>>> and PULL replicas and test with those.
> >>>>
> >>>> For this patch to be complete, we should either extend that test or
> >>>> make another that fails without this patch and succeeds with it.
> >>>>
> >>>> I'd probably recommend modifying TestRebalanceLeaders to randomly
> >>>> create TLOG and (maybe) PULL replicas so we'd keep covering the
> >>>> various cases.
> >>>>
> >>>> Best,
> >>>> Erick
> >>>>
> >>>>
> >>>> On Thu, Dec 20, 2018 at 8:06 AM Bernd Fehling
> >>>> <be...@uni-bielefeld.de> wrote:
> >>>>>
> >>>>> Hi Vadim,
> >>>>> I just tried it with 6.6.5.
> >>>>> In my test cloud with 5 shards, 5 nodes, 3 cores per node it missed
> >>>>> one shard to become leader. But noticed that one shard already was
> >>>>> leader. No errors or exceptions in logs.
> >>>>> May be I should enable debug logging and try again to see all logging
> >>>>> messages from the patch.
> >>>>>
> >>>>> Might be they also changed other parts between 6.6.5 and 7.6.0 so that
> >>>>> it works for you.
> >>>>>
> >>>>> I also just changed from zookeeper 3.4.10 to 3.4.13 which works fine,
> >>>>> even with 3.4.10 dataDir. No errors no complains. Seems to be compatible.
> >>>>>
> >>>>> Regards, Bernd
> >>>>>
> >>>>>
> >>>>> Am 20.12.18 um 12:31 schrieb Vadim Ivanov:
> >>>>>> Yes! It works!
> >>>>>> I have tested RebalanceLeaders today with the patch provided by Endika Posadas. (http://lucene.472066.n3.nabble.com/Rebalance-Leaders-Leader-node-deleted-when-rebalancing-leaders-td4417040.html)
> >>>>>> And at last it works as expected on my collection with 5 nodes and about 400 shards.
> >>>>>> Original patch was slightly incompatible with 7.6.0
> >>>>>> I hope this patch will help to try this feature with 7.6
> >>>>>> https://drive.google.com/file/d/19z_MPjxItGyghTjXr6zTCVsiSJg1tN20
> >>>>>>
> >>>>>> RebalanceLeaders was not very useful feature before 7.0 (as all replicas were NRT)
> >>>>>> But new replica types made it very helpful to keep big clusters in order...
> >>>>>>
> >>>>>> I wonder, why there is no any jira about this case (or maybe I missed it)?
> >>>>>> Anyone who cares, please, help to create jira and improve this feature in the nearest releaase
> >>>>>>