Created a JIRA ticket to track this. https://issues.apache.org/jira/browse/BOOKKEEPER-400
On Tue, Sep 11, 2012 at 11:25 AM, Flavio Junqueira <[email protected]>wrote: > You should be able to upload it to a jira. I think it is best to create a > jira in any case and have a discussion there. > > -Flavio > > On Sep 11, 2012, at 6:55 PM, Aniruddha Laud wrote: > > I'm not able to send the log files (3.7 MB in size) to the dev mailing > list. Any other address that I could send it to? > > Regards, > Aniruddha. > > On Sun, Sep 9, 2012 at 8:14 PM, Sijie Guo <[email protected]> wrote: > >> Hi Aniruddha: >> >> > Yes. What I wanted to say was that if you consider a ledger L and it's >> entries on bookie B, B would have entries E1, E2, E3, E5, E6, E7, E9 and >> so >> on.. >> >> Is it possible to provide the detail metadata of the Ledger L? you could >> replace the detail hostnames with trivial characters. >> >> Besides that, what kind of log level for your hub server? If it is DEBUG, >> I >> think it would have logs about the metadata changes of that Ledger. Could >> you take a look at it? If it is OK, you could share them to help finding >> what happens to that ledger. >> >> > I don't quite understand why the ensemble would change unless we got an >> invalid response from the bookie. >> >> A read request might be error out when read timeout, which causes a >> ensemble change. Is there any more detail log about that entry? >> >> -Sijie >> >> On Sun, Sep 9, 2012 at 6:53 AM, Aniruddha Laud <[email protected] >> >wrote: >> >> > Hi Flavio, >> > >> > Please find my responses inline. >> > >> > On Sat, Sep 8, 2012 at 1:05 AM, Flavio Junqueira <[email protected]> >> > wrote: >> > >> > > Hi Aniruddha, >> > > >> > > On Sep 8, 2012, at 4:55 AM, Aniruddha Laud wrote: >> > > >> > > > One of our hedwig hubs was stuck on reading a particular entry from >> > > > bookkeeper. The entry it was trying to read did not exist in any of >> the >> > > > bookies from the ensemble responsible for that entry (ensemble >> > > information >> > > > obtained from the zookeeper entry). However, some other bookies that >> > were >> > > > not in that ensemble did have that entry. >> > > >> > > I suppose that you ran into crash scenarios. Otherwise I'm not sure >> how >> > > bookies outside the ensemble could have the entry. Is this correct? >> > > >> > The hubs didn't restart while this was going on. Neither did the >> bookies. >> > I'm not very sure why they were sending invalid responses which >> resulted in >> > the handleBookieFailure function being invoked. >> > >> > > >> > > > >> > > > We have a quorum size of 3 and ensemble size of 4, so the expected >> > > behavior >> > > > would be for every 4th entry to be absent for a ledger on any bookie >> > from >> > > > that ensemble. >> > > >> > > Say you have b1, b2, b3, b4 as bookies of your ledger ensemble. Every >> 4th >> > > entry should be stored on b1, b2, b4 if your write quorum has size 3. >> Do >> > > you agree? >> > > >> > Yes. What I wanted to say was that if you consider a ledger L and it's >> > entries on bookie B, B would have entries E1, E2, E3, E5, E6, E7, E9 >> and so >> > on.. >> > >> > > >> > > > However, this was not the case. Some bookies had gaps >> > > > greater than 1 for that ledger, while in some places, the gap was 0. >> > The >> > > > ensemble was changed for the same ledger-id, start-entry-id pair >> many >> > > times >> > > > (around 25) over a period of 14 minutes. >> > > > >> > > >> > > It is not clear if the changes to the ensemble were induced or if the >> > > ensemble was changing without any apparent problem. The behavior seems >> > > awkward, though, and it would be great to see some logs. >> > > >> > I don't quite understand why the ensemble would change unless we got an >> > invalid response from the bookie. >> > >> > > >> > > > After the last "Unsetting success for ledger ... " message from >> > > > PendingAddOp for that particular (ledger, entry) pair, the ensemble >> > > changes >> > > > at least 2 times with the same startEntryId, but we don't see any >> > > > "Unsetting success messages". >> > > >> > > Is this the same entry you couldn't read? >> > > >> > Yes, it's the same entry. >> > >> > > >> > > > >> > > > None of the fields from LedgerHandle or PendingAddOp are thread >> safe, >> > > yet, >> > > > it seems that they could be accessed from different threads. For >> > example, >> > > > it seems like PendingAddOp#writeComplete is called from a different >> > > thread >> > > > than PendingAddOp#unsetSuccessAdnSendWriteRequest. >> > > > >> > > >> > > You might be right here. These two methods seem to be called from >> > > different threads and they both update numResponsesPending, which is >> just >> > > an int. We need to look into this further. >> > > >> > > > I took a look at BOOKKEEPER-337 but I'm not sure if that fixes this. >> > Does >> > > > it? >> > > >> > > It is still a bit unclear what the problem is, so I can't really tell. >> > > >> > > > >> > > > Any insight would be helpful. Also, is there any way to recover from >> > this >> > > > :) ? >> > > >> > > If the entry still exists in a bookie, it is possible to read directly >> > > from a bookie and recover data. I believe Sijie developed some tools >> for >> > > this, but I can't find the jira number now. >> > > >> > Thanks. I'll take a look at the recovery tools. >> > >> > > >> > > -Flavio >> > > >> > > >> > > Thanks for helping :) >> > >> > Regards, >> > Aniruddha. >> > >> > > >
