You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by "Brahma Reddy Battula (JIRA)" <ji...@apache.org> on 2012/07/11 16:04:35 UTC

[jira] [Created] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Brahma Reddy Battula created BOOKKEEPER-337:
-----------------------------------------------

             Summary: Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
                 Key: BOOKKEEPER-337
                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
             Project: Bookkeeper
          Issue Type: Bug
            Reporter: Brahma Reddy Battula


Scenario:
========
Start Five BK's
Write ledger's with ensemble three and quroum size=2
while write inprogress down two bookies(Bookies should be in ensemble)


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13434057#comment-13434057 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

@Ivan 
It would be great if you could also review the proposed changes and push the issue in.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Sijie Guo (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13430880#comment-13430880 ] 

Sijie Guo commented on BOOKKEEPER-337:
--------------------------------------

seems that the patch is based my last patch just changing currentStartEntryId check to state check, right? 

state check is missed in my last patch, which is necessary. as you described, there would be only one ensemble used to write at that time, checking currentStartEntryId is trivial. so I am OK with this change.


                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Sijie Guo (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13414637#comment-13414637 ] 

Sijie Guo commented on BOOKKEEPER-337:
--------------------------------------

yup, Ivan's patch is quite simple and straightforward. but as Rakesh mentioned in previous comment, #handleBookieFailure would be called concurrently to recover same ensemble. so metadata.currentEnsemble would be changed several times before the write callback executed.

suppose, we had ensemble [ A, B, C, D, E ]. A, B, C are down, F, G, H is available.
A -> handleBookieFailure to replace A with F. [F, B, C, D, E]. write request 1 is sent to zookeeper.
B -> handleBookieFailure to replace B with G. [F, G, C, D, E]. write request 2 is sent to zookeeper.
C -> handleBookieFailure to replace C with H. [F, G, H, D, E]. write request 3 is sent to zookeeper.

write request 1 is back. succeed.
write request 2 should fail due to metadata version conflict. so it would call #handleBookieFailure again, but its current ensemble is [F, G, H, D, E], there is no other bookie could be picked up to replace G.

the problem here is we don't have any synchronization on metadata.currentEnsemble. so we got confused when metadata version conflicts. so we had to have synchronized on metadata.currentEnsemble to avoid correct bookie replacement is modified by wrong replacement. also we not only check new metadata's currentEnsemble but also need to check current metadata's currentEnsemble to see whether that bookie has been replaced before. if it has been replaced before, we should not do #handleBookieFailure again.

attach a new patch based on Ivan's patch and Rakesh's test case.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13430884#comment-13430884 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

@Sijie
Thanks for your time and review.
Yup, I've rebased the patch as mentioned in my above comments. Also added one more test case with async writes.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Resolved] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Ivan Kelly (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ivan Kelly resolved BOOKKEEPER-337.
-----------------------------------

    Resolution: Fixed

Committed r1378023. Good work Rakesh.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.2.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Ivan Kelly (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13436832#comment-13436832 ] 

Ivan Kelly commented on BOOKKEEPER-337:
---------------------------------------

{code}
// Update ledger metadata in zk, if in-memory metadata doesn't contains
// the failed bookie.
if (!metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
    LOG.info("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()));
    writeLedgerConfig(new ChangeEnsembleCb());
} else {
    handleBookieFailure(addr, bookieIndex);
}
{code}
I don't think the else part of this can ever be reached. Could you give an example of how it could be?

Also, it would be good to move define the callback for rereadMetadata in the same way as ChangeEnsembleCb as the nesting is going a bit too deep.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Brahma Reddy Battula (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13411538#comment-13411538 ] 

Brahma Reddy Battula commented on BOOKKEEPER-337:
-------------------------------------------------

I had faced this in HDFS-HA,where ANN got shutdown by throwing following..Here availoble bookies are seven even then addentry failed because continuous fail bookie fail which are in ensemble
{noformat}
2012-07-11 04:02:50,445 ERROR hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Invoked callback method: 1058
2012-07-11 04:02:50,445 ERROR hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Could not write request for adding entry: 1058 ledger-id: 364 bookie: /HOST****.168:3183
2012-07-11 04:02:50,445 WARN hidden.bkjournal.org.apache.bookkeeper.client.PendingAddOp: Write did not succeed: 364, 1058
2012-07-11 04:02:50,445 ERROR hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Invoked callback method: 1058
2012-07-11 04:02:50,445 ERROR hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Unexpected add response received from bookie: /HOST****.168:3183 for ledger: 364, entry: 1058 , ignoring
2012-07-11 04:02:50,445 ERROR hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Unexpected add response received from bookie: /HOST****.168:3182 for ledger: 364, entry: 1058 , ignoring
2012-07-11 04:02:50,981 ERROR hidden.bkjournal.org.apache.bookkeeper.client.LedgerHandle: Could not resolve ledger metadata conflict while changing ensemble to: [/HOST****.20:3181, /HOST****.168:3181, /10.18.52.55:3187], old meta data is 
BookieMetadataFormatVersion	1
2
3
0
0	HOST****.168:3183	HOST****.168:3181	HOST****.168:3182
, new meta data is 
BookieMetadataFormatVersion	1
2
3
0
0	HOST****.168:3183	HOST****.168:3181	HOST****.168:3182
1058	HOST****.168:3183	HOST****.168:3181	10.18.52.55:3187
 ,closing ledger
2012-07-11 04:02:51,366 FATAL org.apache.hadoop.hdfs.server.namenode.FSEditLog: Error: flush failed for required journal (JournalAndStream(mgr=org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager@f73f2c5, stream=org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogOutputStream@5476b70))
java.io.IOException: Failed to write to bookkeeper; Error is (-9) Error while using ZooKeeper
	at org.apache.hadoop.contrib.bkjournal.BookKeeperEditLogOutputStream.flushAndSync(BookKeeperEditLogOutputStream.java:141)
	at org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream.flush(EditLogOutputStream.java:105)
	at org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalSetOutputStream$8.apply(JournalSet.java:460)
	at org.apache.hadoop.hdfs.server.namenode.JournalSet.mapJournalsAndReportErrors(JournalSet.java:319)
	at org.apache.hadoop.hdfs.server.namenode.JournalSet.access$200(JournalSet.java:50)
	at org.apache.hadoop.hdfs.server.namenode.JournalSet$JournalSetOutputStream.flush(JournalSet.java:456)
	at org.apache.hadoop.hdfs.server.namenode.FSEditLog.logSync(FSEditLog.java:550)
	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:2007)
	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:471)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.addBlock(ClientNamenodeProtocolServerSideTranslatorPB.java:292)
	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java:42676)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:427)
	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:916)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1692)
	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1688)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:396)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1232)
	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1686)
{noformat}
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13443040#comment-13443040 ] 

Hudson commented on BOOKKEEPER-337:
-----------------------------------

Integrated in bookkeeper-trunk #676 (See [https://builds.apache.org/job/bookkeeper-trunk/676/])
    BOOKKEEPER-337: Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures (rakeshr via ivank) (Revision 1378023)

     Result = SUCCESS
ivank : 
Files : 
* /zookeeper/bookkeeper/trunk/CHANGES.txt
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java

                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.2.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Sijie Guo (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sijie Guo updated BOOKKEEPER-337:
---------------------------------

    Attachment: BOOKKEEPER-337-try2.patch
    
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Sijie Guo (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Sijie Guo updated BOOKKEEPER-337:
---------------------------------

    Attachment: BOOKKEEPER-337-try.patch

yup, ur right. the confliction occurred due to there are more than two changing-same-ensemble requests sending on-the-fly. but LedgerMetadata#resolveConflict doesn't handle this case. so it threw the MetadataVersionException. 

I think the simple fix would be improve LedgerMetadata#resolveConflict to handle this case. I attached a draft about it and also add some logging messages to inform users that there are more than two changing-same-ensemble requests are processing.

but I don't think the fix is best. since it introduced overhead to maintain confliction cases that we need to handle in LedgerMetadata#resolveConflict. for long term plan, seems that we need to find better solution to resolve confliction. 
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13430870#comment-13430870 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

@Ivan @Sijie
Thanks a lot for your thoughts. I've modified the logic little bit, please review.

Latest patch contains:
# synchronized block while replacing the failed bookie in the ensemble. This is done to avoid concurrent ensemble modification (same as mentioned in the above discussion threads)
# When there is a zversion conflict, 
 - first would check the metadata state, return false and throws exception if the state mismatches
 - update znode version
 - then if my changes doesn't exists in zk, writeLedgerConfig(), otw again back to handleFailure
 - return success and continue with write req, if in-memory structure is insync with zk.


@Sijie
I'm thinking the resolveconflicts() would not required to have the currentStartEntryId checks. Instead it can be generic by only checking the states. Any thoughts?

                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13413703#comment-13413703 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

@Sijie
Is there any chance of multiple-thread execution in the following snippet? (I'm just doubting, since all these callbacks are getting invoked upon netty server notifications)
{code}BookieClient.addEntry(args....){
final PerChannelBookieClient client = lookupClient(addr);
        client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
            @Override
            public void operationComplete(int rc, Void result) {
                if (rc != BKException.Code.OK) {
                    cb.writeComplete(rc, ledgerId, entryId, addr, ctx);
                    return;
                }
                client.addEntry(ledgerId, masterKey, entryId, toSend, cb, ctx, options);
            }
        });
}
{code}

Assume we have ensemble like 0 A B C D E
Say, A B C are down in a row and now reached the callback of A's failure, sametime B's connect failure, then sametime C's connect failure?
If yes, all three guys will do modifying the ledger metadata(ensemble) at the sametime and will clash eachother. Any thoughts?
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Ivan Kelly (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ivan Kelly updated BOOKKEEPER-337:
----------------------------------

    Attachment: BOOKKEEPER-337.diff

The solution has to be at the handleBookieFailure level and not the resolveConflict level, because otherwise it could try replace both failed bookie with the same new bookie. I think it's quite straightforward though. 

Suggested fix attached.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Rakesh R reassigned BOOKKEEPER-337:
-----------------------------------

    Assignee: Rakesh R
    
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Rakesh R updated BOOKKEEPER-337:
--------------------------------

    Attachment: BOOKKEEPER-337.1.patch
    
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13440970#comment-13440970 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

Hi Sijie,

Its great and will help us to shape our thoughts on the following case. I hope you were also thinking about handlingBookieFailure?
Also, could you please check my previous comment to get more on my view.

bq.I don't think the else part of this can ever be reached.
{code}
if (!metadata.currentEnsemble.get(bookieIndex).equals(addr)) {

} else {
    handleBookieFailure(addr, bookieIndex);
}
{code}

Thanks,
Rakesh
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13412531#comment-13412531 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

I just tried the following test case by putting break points in the AbstractZkLedgerManager.writeLedgerMetadata()

Consider the case, where added 5 entries and now the writer is idle.
Ledger metadata as follows: 0 - A B C
Say bookies A and B got shutdown. Now, when the writer tries to add more entries it will try reforming the ensemble for A in first pass:
0  - A B C
5 - F B C
Then in the second pass when tries to reform the ensemble for B, it is throwing ZK BadVersionException and not able to resolve the conflicts.

----

??I think the cause is simple, +due to the async setdata+, on handleBookieFailure() its just passing the request to zk and moving to process next ChannelBookieClient response. As the zk callback didn't responsed, this client response seeing the previous zkVersion and is causing the trouble.??

AbstractZkLedgerManager.java
{code}
public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata,
                                    final GenericCallback<Void> cb) {
        //.....
        //.....
        zk.setData(getLedgerPath(ledgerId),
                   metadata.serialize(), zv.getZnodeVersion(),
                   new StatCallback() {
{code}

It would be great to know the opinion from others also.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13438490#comment-13438490 ] 

Rakesh R commented on BOOKKEEPER-337:
-------------------------------------

Thanks a lot Ivan for the reviews.

@Ivan
bq.I don't think the else part of this can ever be reached. Could you give an example of how it could be?

Yeah you're correct, there is no straight away scenario. What I referred is, LedgerHandle#recover() is directly replacing "metadata = newMeta;". (Will come only if recovery and write calls from same bkclient)

{code}
synchronized void recover(final GenericCallback<Void> cb) {
  public void operationComplete(int rc, LedgerMetadata newMeta) {
     if (rc != BKException.Code.OK) {
         cb.operationComplete(rc, null);
     } else {
         metadata = newMeta;
         recover(cb);
      }
  }
{code}

Also, I know its a very corner case and I just included it as defensive coding.

bq.Also, it would be good to move define the callback for rereadMetadata in the same way as ChangeEnsembleCb as the nesting is going a bit too deep

Sure. I'll refactor it.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>    Affects Versions: 4.0.0
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>             Fix For: 4.2.0
>
>         Attachments: BOOKKEEPER-337.1.patch, BOOKKEEPER-337.diff, BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try2.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (BOOKKEEPER-337) Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures

Posted by "Rakesh R (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/BOOKKEEPER-337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Rakesh R updated BOOKKEEPER-337:
--------------------------------

    Attachment: BOOKKEEPER-337-testtoreproduce.patch

Attached the test case to reproduce the issue.
                
> Add entry fails with MetadataVersionException when last ensemble has morethan one bookie failures
> -------------------------------------------------------------------------------------------------
>
>                 Key: BOOKKEEPER-337
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-337
>             Project: Bookkeeper
>          Issue Type: Bug
>            Reporter: Brahma Reddy Battula
>            Assignee: Rakesh R
>         Attachments: BOOKKEEPER-337-testtoreproduce.patch, BOOKKEEPER-337-try.patch
>
>
> Scenario:
> ========
> Start Five BK's
> Write ledger's with ensemble three and quroum size=2
> while write inprogress down two bookies(Bookies should be in ensemble)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira