Hi Flavio, If AutoReplicationWork completes and stabilized, In the clusters where we enabled autoRecovery will not require of recoveryTool any more.
Since we are making autorecovery as optional currently, some users may want to run that RecoveryTool ondemand and may not enable autoRecovery. Once we complete and stabilize the AutoReplicationWork, may be we can go for vote? Regards, Uma ________________________________________ From: Flavio Junqueira [[email protected]] Sent: Thursday, June 28, 2012 7:20 PM To: [email protected] Subject: Re: Race condition between LedgerChecker and Ensemble reformation from client That's cool. I'm still wondering about the bookie recovery tool. Is there still a need for such a tool or the replication scheme will supersede it? What's your opinion? -Flavio On Jun 28, 2012, at 10:08 AM, Uma Maheswara Rao G wrote: > Yes, Flavio there is no duplication of code. > Fragment replication part I moved from BKAdmin to LedgerFragmentReplicator > class.See initial patch at BK-299. > This is just like a helper class for BKadmin for replicating Fragment. > Directly I used this FragmentReplicator part for Replication Wroker. > > Regards, > Uma > ________________________________________ > From: Flavio Junqueira [[email protected]] > Sent: Thursday, June 28, 2012 3:10 AM > To: [email protected] > Cc: Ivan Kelly; Rakesh R > Subject: Re: Race condition between LedgerChecker and Ensemble reformation > from client > > This discussion made me wonder about the relation between the bookie recovery > tool and the auto-recovery feature. Does the latter replace the former? Also, > if they share code, we want to avoid duplication, yes? > > -Flavio > > On Jun 27, 2012, at 4:17 PM, Uma Maheswara Rao G wrote: > >> Thanks Ivan and Flavio. >> >> I got the point. >> >> >> >> And Yes, I have seen this with below mentioned steps. Infact, I used only >> very lower part of the code from BKAdmin.(Only fragment replication part) >> That will not have this prventing steps because that is only responsible for >> fragment. >> >> Need to build in ReplicationWorker. >> >> You mean that, >> >>> 1. If the failed bookie is not in the last ensemble of the ledger, >> recover as normal. >> fine. >> >> 2. If the failed bookie is in the last ensemble of the ledger, we >> reopen the ledger using fencing. This stops the client from writing >> any further entries to the ledger. Then recovery can continue as if >> the ledger had already been closed. >> This can make the NN to switch right? >> >> I think we should have some delay for replication work to trigger. Otherwise >> every ensemble change may enable RW to fence the ledger right? Infact >> session timeout should help here. though there is an other case where delay >> will not help. Ledger already marked as UR bacause of some BK in previous >> enseble. That can trigger RW to scan ledger to find fragments. In my case we >> are keep shutting down the BKs and starting after some time. >> >> >> >> Regards, >> >> Uma >> >> ________________________________________ >> From: Flavio Junqueira [[email protected]] >> Sent: Wednesday, June 27, 2012 7:15 PM >> To: [email protected] >> Cc: Ivan Kelly; Rakesh R >> Subject: Re: Race condition between LedgerChecker and Ensemble reformation >> from client >> >> Hi Uma, Check the whole paragraph: Consequently, we restrict the recovery >> tool to only perform changes to the metadata when the ledger is closed or >> when the ledger writer has detected the bookie crash, has replaced it, and >> reflected the change in the metadata. >> >> It is not only for closed ledgers. >> >> -Flavio >> >> On Jun 27, 2012, at 1:40 PM, Uma Maheswara Rao G wrote: >> >>> Thanks a lot, Flavio for reference. >>> >>> Here we are making use of RecoveryTool code. >>> >>> Also I have seen in the doc saying: >>> Consequently, we restrict the recovery tool to only perform changes to the >>> metadata when >>> the ledger is closed >>> >>> >>> >>> In BOOKKEEPER-112 , Client is trying to handle this metadat failure case. >>> But still there is a case it can not handle. >>> >>> Here is the case : >>> >>> When one BK failed from ensemble it will try to update the ensemble >>> with new BK. >>> >>> >>> >>> CLIENT STEP 1: ex: 10 x y z -->10 x a z >>> >>> >>> >>> BETWEEN Step1 and Between Step2: >>> >>> At this stage , If RT runs, it may thing that there is missed entry, >>> because a does not have the entry written yet. It may replace with new BK >>> again by copying that missed entry. >>> >>> AutoRT updated ensemble ----> 10 x b z >>> >>> >>> >>> >>> >>> CLINET STEP2: And start writing the failed entry to pending BKs, >>> unfortunately again it will try to update ensemble, but whatver ensemble >>> knows by client is '10 x a z' >>> >>> >>> >>> Now metadata updation should fail as it got changed RT. >>> >>> >>> >>> In this case resolve conflicts obiously can not be solved. will be closed as >>> >>> 10 x b z >>> >>> 9 CLOSED >>> >>> >>> >>> Falvio, Ivan and Sijie What about your opinion on this case? >>> >>> >>> >>> >>> >>> Should be ok to skip OPENED ledgers? as standby will do rolling for every 2 >>> mins. So, 2mins data may be in OPENED ledger. >>> >>> Let's check for other scenarios as well. >>> >>> >>> >>> >>> >>> Regards, >>> >>> Uma >>> >>> >>> >>> ________________________________________ >>> From: Flavio Junqueira [[email protected]] >>> Sent: Wednesday, June 27, 2012 12:15 PM >>> To: [email protected] >>> Cc: Ivan Kelly; Rakesh R >>> Subject: Re: Race condition between LedgerChecker and Ensemble reformation >>> from client >>> >>> Hi Uma, We have had a related issue in BOOKKEEPER-112 and there is a doc >>> there describing how we deal with it. It might help to give it a look. >>> >>> -Flavio >>> >>> On Jun 27, 2012, at 7:06 AM, Uma Maheswara Rao G wrote: >>> >>>> Right. But Current Replication process considered for OPEN ledgers also. >>>> So, Ledger checker can not know whether that ensemble is just reformed by >>>> client or inprogress for write. >>>> >>>> One way is to skip the replication for Inprogress Ledgers. But Auditor may >>>> need to recheck this opened ledgers periodically which ever it came across? >>>> >>>> IMO, replicating inrprogress ledgers may create some inconsistencies. >>>> >>>> Thanks, >>>> Uma >>>> ________________________________________ >>>> From: Flavio Junqueira [[email protected]] >>>> Sent: Wednesday, June 27, 2012 4:21 AM >>>> To: [email protected] >>>> Cc: Ivan Kelly; Rakesh R >>>> Subject: Re: Race condition between LedgerChecker and Ensemble >>>> reformation from client >>>> >>>> Hi Uma, It sounds like the replication worker shouldn't have written: >>>> >>>> 401 10.18.40.155:3181 10.18.40.155:3185 >>>> 10.18.40.155:3184 >>>> >>>> If I'm not missing anything, the replication worker should update an >>>> existing entry in the metadata, not create a new entry. >>>> >>>> -Flavio >>>> >>>> On Jun 26, 2012, at 6:07 PM, Uma Maheswara Rao G wrote: >>>> >>>>> Hi, >>>>> >>>>> It looks there is a race between LedgerChecker and Ensemble reformation >>>>> from client. >>>>> >>>>> When one bookie failed from ensemble quoram, it will try to reform the >>>>> ensemble on handleBookieFailure. >>>>> >>>>> At this time it is reforming the ensemble and resending the write request >>>>> to new bookie (which is added into new ensemble.) >>>>> >>>>> At the same time if, If ReplicationWroker triggers on same ledger and run >>>>> the LedgerChecker on it. >>>>> LedgerChecker may find this last failed entry also as a fragment, because >>>>> ensemble change already updated in metadata. >>>>> >>>>> If ReplicationWorker replicate this last fragment, then >>>>> ChangeEnsembleCb#operationComplete will fail with Badversion, because >>>>> ensemble data already updated by ReplicationWorker. >>>>> >>>>> >>>>> LOG.error("Could not resolve ledger metadata conflict while changing >>>>> ensemble to: " >>>>> + newEnsemble + ", old >>>>> meta data is \n" + new String(metadata.serialize()) >>>>> + "\n, new meta data is >>>>> \n" + new String(newMeta.serialize()) + "\n ,closing ledger"); >>>>> >>>>> 2012-06-23 10:51:47,814 - ERROR >>>>> [main-EventThread:LedgerHandle$1ChangeEnsembleCb$1$1@714] - Could not >>>>> resolve ledger metadata conflict while changing ensemble to: >>>>> [/10.18.40.155:3182, /10.18.40.155:3185, /10.18.40.155:3184], old meta >>>>> data is >>>>> BookieMetadataFormatVersion 1 >>>>> 2 >>>>> 3 >>>>> 0 >>>>> 0 10.18.40.155:3181 10.18.40.155:3182 >>>>> 10.18.40.155:3183 >>>>> 102 10.18.40.155:3181 10.18.40.155:3185 >>>>> 10.18.40.155:3183 >>>>> , new meta data is >>>>> BookieMetadataFormatVersion 1 >>>>> 2 >>>>> 3 >>>>> 0 >>>>> 0 10.18.40.155:3181 10.18.40.155:3182 >>>>> 10.18.40.155:3183 >>>>> 102 10.18.40.155:3181 10.18.40.155:3185 >>>>> 10.18.40.155:3183 >>>>> 401 10.18.40.155:3181 10.18.40.155:3185 >>>>> 10.18.40.155:3184 >>>>> ,closing ledger >>>>> >>>>> >>>>> After this time, it will close the ledger. >>>>> asyncCloseInternal(NoopCloseCallback.instance, null, rc); >>>>> >>>>> Then finally ledger metadata will looks like: >>>>> >>>>> 0 10.18.40.155:3181 10.18.40.155:3182 >>>>> 10.18.40.155:3183 >>>>> 102 10.18.40.155:3181 10.18.40.155:3185 >>>>> 10.18.40.155:3183 >>>>> 401 10.18.40.155:3181 10.18.40.155:3185 >>>>> 10.18.40.155:3184 >>>>> 400 CLOSED >>>>> >>>>> Because client known last succussful entry is 400. Am i missing some >>>>> thing here? >>>>> >>>>> >>>>> >>>>> >>>>> >>>>> Regards, >>>>> >>>>> Uma >>>>> >>>>> >>>>> >>>>>
