You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by "Philipp Sushkin (Created) (JIRA)" <ji...@apache.org> on 2012/02/07 06:16:59 UTC

[jira] [Created] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

LedgerHandle.readLastConfirmed does not work
--------------------------------------------

                 Key: BOOKKEEPER-162
                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
             Project: Bookkeeper
          Issue Type: Bug
          Components: bookkeeper-client
    Affects Versions: 4.0.0
            Reporter: Philipp Sushkin
            Priority: Critical


Two bookkeeper clients.
1st continuously writing to ledger X.
2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.

In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).

Main question - is given scenario is expected to work at all?



--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

About the point on readLastConfirmed, I don't think that's a bug. This call gets the hints from the bookies. There is a similar call that simply reads the local value of the ledger handle once the ledger is open.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Yes, we write lastAddConfirmed upon every add. No bookie alone knows whether an add has been confirmed to the client application or not.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin resolved BOOKKEEPER-162.
----------------------------------------

       Resolution: Not A Problem
    Fix Version/s: 4.0.0
    
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin updated BOOKKEEPER-162:
---------------------------------------

    Attachment: BookieReadWriteTest.java.patch

Same patch, but reverted constructor change (I don't use Parametrized test, ofcourse).
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira updated BOOKKEEPER-162:
----------------------------------------

    Attachment: BOOKKEEPER-162.patch

Thanks, Sijie. Not using max was part of debugging. I've put it back.

The reference to LedgerHandle in PerChannelBookieCliet is due to the constant LAST_ADD_CONFIRMED that I moved to BookieProtocol. I think we forgot to move it when we wrote BookieProtocol. To me at least it makes more sense there, though.

The reference to BookKeeperTestClient was a leftover from Philipp's patch that I forgot to remove when I cleaned it up.

I'm attaching a new patch that fixes all these.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Agreed, it shouldn't return 0 when the ledger is empty. In the case the ledger has one element, it should still return -1 (empty) according to the semantics of the call. readLastConfirmed returns the maximum hint across all bookies, and the hint for a ledger is the value of the last confirmed field in the last entry it wrote. Consequently, if there is only one entry written, the hint will say that there is no add confirmed before that one, which is correct.

Perhaps if you need to know precisely which entries have been confirmed, you may want to have the writer communicating to the readers though ZooKeeper or directly (e.g., TCP). The readLastConfirmed mechanism gives an approximation of the state of the ledger, and is particularly useful when writing streams continuously. 

If you can say more about your use case, we may be able to help you decide, Philipp.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira updated BOOKKEEPER-162:
----------------------------------------

    Attachment: BOOKKEEPER-162.patch

I have refined the test a bit. 
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira updated BOOKKEEPER-162:
----------------------------------------

    Attachment: BOOKKEEPER-162.patch

Fixed the initialization problem.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:51 PM:
--------------------------------------------------------------------

I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code:title=BookieServer.java|borderStyle=solid}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself
{code:title=EntryLogger.java|borderStyle=solid}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...
    }
{code}
I see data itself
{panel}
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}

On client in 
{code:title=DigestManager.java|borderStyle=solid}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 32 after "dataReceived.readerIndex(8);" - *adjustm. 24 + 8)
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                
      was (Author: philipp.sushkin):
    I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code:title=BookieServer.java|borderStyle=solid}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself
{code:title=EntryLogger.java|borderStyle=solid}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 
{code:title=DigestManager.java|borderStyle=solid}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 32 after "dataReceived.readerIndex(8);" - *adjustm. 24 + 8)
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:46 PM:
--------------------------------------------------------------------

I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 32 after "dataReceived.readerIndex(8);" - *adjustm. 24 + 8)
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                
      was (Author: philipp.sushkin):
    I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 24 after "dataReceived.readerIndex(8);")
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13203255#comment-13203255 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/8/12 5:37 AM:
--------------------------------------------------------------------

So my problem - using async adds. That is why LedgerHandle.readLastConfirmed returns 0.
In fact, as I understand, it is not correct anyway. It should be -1 in this case (everybookie returned -1) ...

For my particular problem - will try to to send empty messages periodically just to maintain
LedgerHandle.readLastConfirmed working on readers...

                
      was (Author: philipp.sushkin):
    So my problem is in using async adds. That is why LedgerHandle.readLastConfirmed returns 0.
In fact, as I understand, it is not correct anyway. It should be -1 in this case ...

For my particular problem - will try to to send empty messages periodically just to maintain
LedgerHandle.readLastConfirmed working on clients...

                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira updated BOOKKEEPER-162:
----------------------------------------

    Attachment: BOOKKEEPER-162.patch

Uploading patch that fixes this issue. It includes the test Philipp provided.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin updated BOOKKEEPER-162:
---------------------------------------

    Attachment: BookieReadWriteTest.java.patch

Please find new patch attached, now test failing.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Correct, we currently don't have an api to check if a ledger is closed. We assume that it is communicated from a writer to a reader by an external channel, e.g., through zookeeper. It wouldn't be difficult to have such a call, though. It is simply checking the ledger metadata on zookeeper.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Philipp, Can you generate a patch with your new test, please?
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

BTW, if a ledger is closed, readLastConfirmedOp would return wrong last confirmed.
we should check a ledger when reading last confirmed. if the ledger is closed, we use the close value in ledger metadata as last confirmed directly, we don't need read it again.

I think this is also a bug related to readLastConfirmed. it is better to fix it in this jira.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

hmm. currently readLastConfirmed doesn't work correctly in following case:

1) there is no entries in the ledger, readLastConfirmed returns 0.
2) there is 1 entries in the ledger, readLastConfirmed returns 0.

this issue is caused by giving a wrong initial value (0) for maxAddConfirmed in ReadLastConfirmedOp.

the expected value would be better as below: 

1) there is no entries in the ledger, readLastConfirmed returns -2.
2) there is 1 entry in the ledger, readLastConfirmed returns -1.

what is your opinion, flavio?
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:37 PM:
--------------------------------------------------------------------

I started to debug and do have small question.


Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 24 after "dataReceived.readerIndex(8);")
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  <- ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  <- long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, <- long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                
      was (Author: philipp.sushkin):
    Comment not finished..
                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:48 PM:
--------------------------------------------------------------------

I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code:title=BookieServer.java|borderStyle=solid}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself
{code:title=EntryLogger.java|borderStyle=solid}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 
{code:title=DigestManager.java|borderStyle=solid}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 32 after "dataReceived.readerIndex(8);" - *adjustm. 24 + 8)
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                
      was (Author: philipp.sushkin):
    I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 32 after "dataReceived.readerIndex(8);" - *adjustm. 24 + 8)
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin commented on BOOKKEEPER-162:
--------------------------------------------

>> The patch I provided fixes this issue.

Thanks a lot.

                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

the new patch is good to me. +1
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Check the API docs: http://zookeeper.apache.org/bookkeeper/docs/r4.0.0/apidocs/

I don't mind having more documentation added if it is not clear the distinction. 
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin updated BOOKKEEPER-162:
---------------------------------------

    Attachment: bookkeeper.log

Also reproduced situation where noRecover reader always getting 0 .

Unfortunately, I cannot reproduce it using bookkeeper test, only with my app. Currently trying to investigate where I could fail.

Attaching log:
Writer sends 2 entries with simple content '1', '2'
Reader tries to read lh.readLastConfirmed(), but always getting 0.


                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Reopened] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin reopened BOOKKEEPER-162:
----------------------------------------


I modified test a little bit to make it fail.

Please find failing version. It changed to use 2 different BookKeeperTestClient to simulate 2 different pc (may failing case) more precisely.

{code}
@Test
    public void testReadFromOpenLedger2() throws Exception {
        try {
            BookKeeperTestClient  writerBkc = new BookKeeperTestClient(baseClientConf);
            BookKeeperTestClient  readerBkc = new BookKeeperTestClient(baseClientConf);
            // Create a ledger
            lh = writerBkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = readerBkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    long readLastConfirmed = lhOpen.readLastConfirmed();
                    assertTrue(readLastConfirmed != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}


I am getting it failing, as I understand, because of "lastEntry > lastAddConfirmed" check.
{code:title=LedgerHandle.java|borderStyle=solid}
    public void asyncReadEntries(long firstEntry, long lastEntry,
                                 ReadCallback cb, Object ctx) {
        // Little sanity check
        if (firstEntry < 0 || lastEntry > lastAddConfirmed
                || firstEntry > lastEntry) {
            cb.readComplete(BKException.Code.ReadException, this, null, ctx);
            return;
        }
...
{code}

So looks like recreation of ledger landle is required every read iteration.

Is it expected?



                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

thanks Flavio. most of the patch is good to me, except that you import LedgerHandle in PerChannelBookieClient and BookKeeperTestClient in BookieReadWriteTest, these two classes are not used, and in ReadLastConfirmedOp.java why you don't use Math.max ?
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin updated BOOKKEEPER-162:
---------------------------------------

    Attachment: BookieReadWriteTest.java.patch

Please find patch with test attached.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

seems the document is not so clear. It would better to add more :)
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Hi Philipp, Are the adds to the ledger asynchronous? Are you calling readLastConfirmed after receiving an ack for some add operation?

In principle, it should work. Check testReadFromOpenLedger in BookieReadWriteTest. 

Do you have logs you can share with us, both bookie and client?

                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira updated BOOKKEEPER-162:
----------------------------------------

    Attachment: BOOKKEEPER-162.patch

It is safer to check if the maxAddConfirmed is larger than lastAddConfirmed before setting. I realized that if the ledger writer invokes it, the patch may set lastAddConfirmed incorrectly. 
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

As you pointed out, the current test in trunk reopens the ledger every time, and it passes if you reopen because we set the lastAddConfirmed value to the max value read from bookies. A second read should fail if you keep reading because the lastEntry the call requests may be larger than the lastAddConfirmed, which is not updated. The patch I provided fixes this issue.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin commented on BOOKKEEPER-162:
--------------------------------------------

So my problem is in using async adds. That is why LedgerHandle.readLastConfirmed returns 0.
In fact, as I understand, it is not correct anyway. It should be -1 in this case ...

For my particular problem - will try to to send empty messages periodically just to maintain
LedgerHandle.readLastConfirmed working on clients...

                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202122#comment-13202122 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 7:21 AM:
--------------------------------------------------------------------

Thanks, i tried updated version of testReadFromOpenLedger

{code}
@Test
    public void testReadFromOpenLedger2() throws IOException {
        try {
            // Create a ledger
            lh = bkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    long readLastConfirmed = lhOpen.readLastConfirmed();
                    assertTrue(readLastConfirmed != 0);
                    assertTrue(lhOpen.getLastAddConfirmed() != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}

In fact nothing is done to original test, but

{code}
LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
{code}
and 
{code}
// close read only ledger should not change metadata
lhOpen.close();
{code}
moved out, and 
{code}
long readLastConfirmed = lhOpen.readLastConfirmed();
assertTrue(readLastConfirmed != 0);
assertTrue(lhOpen.getLastAddConfirmed() != 0);
{code}

So looks like it was misusing of lhOpen.getLastAddConfirmed(), should use lhOpen.readLastConfirmed() instead.
Closing issue.

Small question - is there any way to get and updateledger status (closed?) using ledger handle? As I see currently there is no public api for that.


                
      was (Author: philipp.sushkin):
    Please find updated version of testReadFromOpenLedger which fails at described case

{code}
@Test
    public void testReadFromOpenLedger2() throws IOException {
        try {
            // Create a ledger
            lh = bkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    lhOpen.readLastConfirmed();
                    assertTrue(lhOpen.getLastAddConfirmed() != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}



In fact nothing is done to original test, but

{code}
LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
{code}
and 
{code}
// close read only ledger should not change metadata
lhOpen.close();
{code}
moved out, and 
{code}
lhOpen.readLastConfirmed();
assertTrue(lhOpen.getLastAddConfirmed() != 0);
{code}
added, where test if failing now.


                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:37 PM:
--------------------------------------------------------------------

I started to debug and do have small question.

Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 24 after "dataReceived.readerIndex(8);")
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  < ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  < long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, < long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                
      was (Author: philipp.sushkin):
    I started to debug and do have small question.


Same case 1 writer, 1 reader.
Reader tries to request lastAddConfirmed.

On server side I see there is header:
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

built in BookieServer
{code}
    private ByteBuffer buildResponse(int errorCode, byte version, byte opCode, long ledgerId, long entryId) {
        ByteBuffer rsp = ByteBuffer.allocate(24);
        rsp.putInt(new PacketHeader(version, 
                                    opCode, (short)0).toInt());
        rsp.putInt(errorCode);
        rsp.putLong(ledgerId);
        rsp.putLong(entryId);

        rsp.flip();
        return rsp;
    }
{code}

EntryId here -1, because I was requesting last confirmed.
Request looked like
{panel}
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]
{panel}

Also data itself, in EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}

On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see (reader offset 24 after "dataReceived.readerIndex(8);")
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2,
 -1, -1, -1, -1, -1, -1, -1, -1, 
 0, 0, 0, 0, 0, 0, 0, 2,  <- ledgerId
 0, 0, 0, 0, 0, 0, 0, 1,  <- long entryId = dataReceived.readLong();
 -1, -1, -1, -1, -1, -1, -1, -1, <- long lastAddConfirmed = dataReceived.readLong(); *PROBLEM HERE*
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


So, am I right: lastAddConfirmed is a part of record, not calculated on request?

                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin commented on BOOKKEEPER-162:
--------------------------------------------

Please find updated version of testReadFromOpenLedger which fails at described case

{code}
@Test
    public void testReadFromOpenLedger2() throws IOException {
        try {
            // Create a ledger
            lh = bkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    lhOpen.readLastConfirmed();
                    assertTrue(lhOpen.getLastAddConfirmed() != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}



In fact nothing is done to original test, but

{code}
LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
{code}
and 
{code}
// close read only ledger should not change metadata
lhOpen.close();
{code}
moved out, and 
{code}
lhOpen.readLastConfirmed();
assertTrue(lhOpen.getLastAddConfirmed() != 0);
{code}
added, where test if failing now.


                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Sijie Guo commented on BOOKKEEPER-162:
--------------------------------------

hmm, I think it would be better to add more documents in readLastConfirmed to tell the difference between readLastConfirmed and getLastAddConfirmed. otherwise the different return value would make user confused. 
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202122#comment-13202122 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 7:23 AM:
--------------------------------------------------------------------

Thanks, I tried updated version of testReadFromOpenLedger

So looks like it was misusing of lhOpen.getLastAddConfirmed(), should use lhOpen.readLastConfirmed() instead.

*Closing issue.*

{code}
@Test
    public void testReadFromOpenLedger2() throws IOException {
        try {
            // Create a ledger
            lh = bkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    long readLastConfirmed = lhOpen.readLastConfirmed();
                    assertTrue(readLastConfirmed != 0);
                    assertTrue(lhOpen.getLastAddConfirmed() != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}

In fact nothing is done to original test, but

{code}
LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
{code}
and 
{code}
// close read only ledger should not change metadata
lhOpen.close();
{code}
moved out, and 
{code}
long readLastConfirmed = lhOpen.readLastConfirmed();
assertTrue(readLastConfirmed != 0);
assertTrue(lhOpen.getLastAddConfirmed() != 0);
{code}


Small question - is there any way to get and updateledger status (closed?) using ledger handle? As I see currently there is no public api for that.


                
      was (Author: philipp.sushkin):
    Thanks, i tried updated version of testReadFromOpenLedger

{code}
@Test
    public void testReadFromOpenLedger2() throws IOException {
        try {
            // Create a ledger
            lh = bkc.createLedger(digestType, ledgerPassword);
            // bkc.initMessageDigest("SHA1");
            ledgerId = lh.getId();
            LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
            LOG.info("Ledger ID: " + lh.getId());
            for (int i = 0; i < numEntriesToWrite; i++) {
                ByteBuffer entry = ByteBuffer.allocate(4);
                entry.putInt(rng.nextInt(maxInt));
                entry.position(0);

                entries.add(entry.array());
                entriesSize.add(entry.array().length);
                lh.addEntry(entry.array());
                if (i == numEntriesToWrite / 2) {
                    // no recovery opened ledger 's last confirmed entry id is
                    // less than written
                    // and it just can read until (i-1)
                    int toRead = i - 1;
                    long readLastConfirmed = lhOpen.readLastConfirmed();
                    assertTrue(readLastConfirmed != 0);
                    assertTrue(lhOpen.getLastAddConfirmed() != 0);
                    Enumeration<LedgerEntry> readEntry = lhOpen.readEntries(toRead, toRead);
                    assertTrue("Enumeration of ledger entries has no element", readEntry.hasMoreElements() == true);
                    LedgerEntry e = readEntry.nextElement();
                    assertEquals(toRead, e.getEntryId());
                    Assert.assertArrayEquals(entries.get(toRead), e.getEntry());
                    // should not written to a read only ledger
                    try {
                        lhOpen.addEntry(entry.array());
                        fail("Should have thrown an exception here");
                    } catch (BKException.BKIllegalOpException bkioe) {
                        // this is the correct response
                    } catch (Exception ex) {
                        LOG.error("Unexpected exception", ex);
                        fail("Unexpected exception");
                    }

                }
            }
            long last = lh.readLastConfirmed();
            assertTrue("Last confirmed add: " + last, last == (numEntriesToWrite - 2));

            LOG.debug("*** WRITE COMPLETE ***");
            // close ledger
            lh.close();
            // close read only ledger should not change metadata
            lhOpen.close();
        } catch (BKException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to BookKeeper exception");
        } catch (InterruptedException e) {
            LOG.error("Test failed", e);
            fail("Test failed due to interruption");
        }
    }
{code}

In fact nothing is done to original test, but

{code}
LedgerHandle lhOpen = bkc.openLedgerNoRecovery(ledgerId, digestType, ledgerPassword);
{code}
and 
{code}
// close read only ledger should not change metadata
lhOpen.close();
{code}
moved out, and 
{code}
long readLastConfirmed = lhOpen.readLastConfirmed();
assertTrue(readLastConfirmed != 0);
assertTrue(lhOpen.getLastAddConfirmed() != 0);
{code}

So looks like it was misusing of lhOpen.getLastAddConfirmed(), should use lhOpen.readLastConfirmed() instead.
Closing issue.

Small question - is there any way to get and updateledger status (closed?) using ledger handle? As I see currently there is no public api for that.


                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Thanks for reporting and helping, Philipp.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Flavio Junqueira commented on BOOKKEEPER-162:
---------------------------------------------

Thanks, Sijie. If no one else has an issue with this patch, I'll commit it later today.
                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin commented on BOOKKEEPER-162:
--------------------------------------------

I am confused, now it is passing...
Either it happens from time to time, or something was corrupted when I was writing it. 

                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Philipp Sushkin commented on BOOKKEEPER-162:
--------------------------------------------

I started to debug and do have small question.


Same case 1 writer, 1 reader.


reader tries to request lastAddConfirmed.

on server side I see

EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

here I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}
Also there is header
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]

built in 
{code}
{code}


On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see 
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2, <- entryId 
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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] [Issue Comment Edited] (BOOKKEEPER-162) LedgerHandle.readLastConfirmed does not work

Posted by "Philipp Sushkin (Issue Comment Edited) (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/BOOKKEEPER-162?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13202362#comment-13202362 ] 

Philipp Sushkin edited comment on BOOKKEEPER-162 at 2/7/12 1:21 PM:
--------------------------------------------------------------------

Comment not finished..
                
      was (Author: philipp.sushkin):
    I started to debug and do have small question.


Same case 1 writer, 1 reader.


reader tries to request lastAddConfirmed.

on server side I see

EntryLogger
{code}
byte[] readEntry(long ledgerId, long entryId, long location) throws IOException {
...
        byte data[] = new byte[entrySize];
        ByteBuffer buff = ByteBuffer.wrap(data);
        int rc = fc.read(buff, pos);
 ...

here I see 
 [0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
    }
{code}
Also there is header
[1, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2, -1, -1, -1, -1, -1, -1, -1, -1]

built in 
{code}
{code}


On client in 

DigestManager
{code}
    RecoveryData verifyDigestAndReturnLastConfirmed(ChannelBuffer dataReceived) throws BKDigestMatchException {
        verifyDigest(dataReceived);
        dataReceived.readerIndex(8);

        long entryId = dataReceived.readLong();
        long lastAddConfirmed = dataReceived.readLong();
        long length = dataReceived.readLong();
        return new RecoveryData(lastAddConfirmed, entryId);

    }
{code}

I see 
{panel}
[1, 2, 0, 0, 0, 0, 0, 0,
 0, 0, 0, 0, 0, 0, 0, 2, <- entryId 
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 0, 0, 0, 0, 0, 0, 0, 1,
 -1, -1, -1, -1, -1, -1, -1, -1,
 0, 0, 0, 0, 0, 0, 0, 2,
 -16, 41, -22, 25, 26, 93, -80, -40, -80, -15, 7, -36, 72, -52, 13, -54, 39, 117, 12, -56, 2]
{panel}


                  
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.0.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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-162) LedgerHandle.readLastConfirmed does not work

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

Hudson commented on BOOKKEEPER-162:
-----------------------------------

Integrated in bookkeeper-trunk #356 (See [https://builds.apache.org/job/bookkeeper-trunk/356/])
    BOOKKEEPER-162: LedgerHandle.readLastConfirmed does not work (fpj) (Revision 1243462)

     Result = SUCCESS
fpj : 
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/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
* /zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieReadWriteTest.java

                
> LedgerHandle.readLastConfirmed does not work
> --------------------------------------------
>
>                 Key: BOOKKEEPER-162
>                 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-162
>             Project: Bookkeeper
>          Issue Type: Bug
>          Components: bookkeeper-client
>    Affects Versions: 4.0.0
>            Reporter: Philipp Sushkin
>            Assignee: Flavio Junqueira
>            Priority: Critical
>             Fix For: 4.1.0
>
>         Attachments: BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BOOKKEEPER-162.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, BookieReadWriteTest.java.patch, bookkeeper.log
>
>
> Two bookkeeper clients.
> 1st continuously writing to ledger X.
> 2nd (bk.openLedgerNoRecovery) polling ledger X for new entries and reading them.
> In response we always reveiceing 0 as last confirmed entry id (in fact we are receiving -1 from each bookie RecoveryData but then in ReadLastConfirmedOp, but uninitialized "long maxAddConfirmed;" takes priority in Math.max(...).
> Main question - is given scenario is expected to work at all?

--
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