hadoop-zookeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Benjamin Reed <br...@yahoo-inc.com>
Subject Re: BookKeeper Doubts
Date Mon, 19 Jul 2010 22:44:00 GMT
you have concluded correctly.

1) bookkeeper was designed for a process to use as a write-ahead log, so 
as a simplifying assumption we assume a single writer to a log. we 
should be throwing an exception if you try to write to a handle that you 
obtained using openLedger. can you open a jira for that?

2) this is mostly true, there are some exceptions. the creater of a 
ledger can read entries even though the ledger is still being written 
to. we would like to add the ability for a reader to assert the last 
entry in a ledger and read up to that entry, but this is not yet in the 
code.

3) there is one other bug you are seeing, before a ledger can be read, 
it must be closed. as your code shows, a process can open a ledger for 
reading while it is still being written to, which causes an implicit 
close that is not detected by the writer.

this is a nice test case :) thanx
ben

On 07/17/2010 05:02 PM, André Oriani wrote:
> Hi,
>
>
> I was not sure if I had understood the behavior of BookKeeper from
> documentation. So I made  a little program, reproduced below, to see what
> BookKeeper looks like in action. Assuming my code is correct ( you never
> know  when your code has some nasty obvious bugs that only other person than
> you can see ) , I could draw the follow conclusions:
>
> 1) Only the creator can add entries to a ledger, even though you can  open
> the ledger, get a handle and call addEntry on it. No exception is thrown  i.
> In other words, you cannot open a ledger for append.
>
> 2) Readers are able to see only the entries that were added to a ledger
> until someone had opened it for reading. If you want to ensure  readers will
> see all the entries, you must add all entries before any reader attempts to
> read from the ledger.
>
> Could someone please tell me if those conclusions are correct or if I am
> mistaken? In the later case, could that person also tell me what is wrong ?
>
> Thanks a lot for the attention and the patience with this BookKeeper newbie,
> André
>
>
>
>
> package br.unicamp.zooexp.booexp;
>
>
> import java.io.IOException;
>
> import java.util.Enumeration;
>
>
> import org.apache.bookkeeper.client.BKException;
>
> import org.apache.bookkeeper.client.BookKeeper;
>
> import org.apache.bookkeeper.client.LedgerEntry;
>
> import org.apache.bookkeeper.client.LedgerHandle;
>
> import org.apache.bookkeeper.client.BookKeeper.DigestType;
>
> import org.apache.zookeeper.KeeperException;
>
>
> public class BookTest {
>
>
>      public static void main (String ... args) throws IOException,
> InterruptedException, KeeperException, BKException{
>
>          BookKeeper bk = new BookKeeper("127.0.0.1");
>
>          LedgerHandle lh = bk.createLedger(DigestType.CRC32, "123"
> .getBytes());
>
>          long lh_id = lh.getId();
>
>          lh.addEntry("Teste".getBytes());
>
>          lh.addEntry("Test2".getBytes());
>
>          System.out.printf("Got %d entries for lh\n"
> ,lh.getLastAddConfirmed()+1);
>
>
>
>
>          lh.addEntry("Test3".getBytes());
>
>          LedgerHandle lh1 = bk.openLedger(lh_id, DigestType.CRC32, "123"
> .getBytes());
>
>          System.out.printf("Got %d entries for lh1\n"
> ,lh1.getLastAddConfirmed()+1);
>
>          lh.addEntry("Test4".getBytes());
>
>
>          lh.addEntry("Test5".getBytes());
>
>          lh.addEntry("Test6".getBytes());
>
>          System.out.printf("Got %d entries for lh\n"
> ,lh.getLastAddConfirmed()+1);
>
>          Enumeration<LedgerEntry>  seq = lh.readEntries(0,
> lh.getLastAddConfirmed());
>
>          while (seq.hasMoreElements()){
>
>              System.out.println(new String(seq.nextElement().getEntry()));
>
>          }
>
>          lh.close();
>
>
>
>          lh1.addEntry("Test7".getBytes());
>
>          lh1.addEntry("Test8".getBytes());
>
>
>          System.out.printf("Got %d entries for lh1\n"
> ,lh1.getLastAddConfirmed()+1);
>
>
>          seq = lh1.readEntries(0, lh1.getLastAddConfirmed());
>
>          while (seq.hasMoreElements()){
>
>              System.out.println(new String(seq.nextElement().getEntry()));
>
>          }
>
>
>
>          lh1.close();
>
>
>          LedgerHandle lh2 = bk.openLedger(lh_id, DigestType.CRC32, "123"
> .getBytes());
>
>          lh2.addEntry("Test9".getBytes());
>
>
>          System.out.printf("Got %d entries for lh2 \n"
> ,lh2.getLastAddConfirmed()+1);
>
>
>          seq = lh2.readEntries(0, lh2.getLastAddConfirmed());
>
>          while (seq.hasMoreElements()){
>
>              System.out.println(new String(seq.nextElement().getEntry()));
>
>          }
>
>
>          bk.halt();
>
>
>      }
>
> }
>
>
> Output:
>
> Got 2 entries for lh
>
> Got 3 entries for lh1
>
> Got 6 entries for lh
>
> Teste
>
> Test2
>
> Test3
>
> Test4
>
> Test5
>
> Test6
>
> Got 3 entries for lh1
>
> Teste
>
> Test2
>
> Test3
>
> Got 3 entries for lh2
>
> Teste
>
> Test2
>
> Test3
>    


Mime
View raw message