This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new f40a29d  Write DEFERRED_SYNC docs
f40a29d is described below

commit f40a29ddef3586c53fad4b9098a85342390a498b
Author: Enrico Olivelli <[email protected]>
AuthorDate: Mon Jul 30 08:29:32 2018 +0200

    Write DEFERRED_SYNC docs
    
    Write a little documentation about DEFERRED_SYNC WriteFlag and force() API
    
    Author: Enrico Olivelli <[email protected]>
    
    Reviewers: Sijie Guo <[email protected]>
    
    This closes #1571 from eolivelli/fix/bp14-docs
---
 site/docs/latest/api/ledger-api.md | 43 ++++++++++++++++++++++++++++++++++++--
 1 file changed, 41 insertions(+), 2 deletions(-)

diff --git a/site/docs/latest/api/ledger-api.md 
b/site/docs/latest/api/ledger-api.md
index 51a6e84..b6cb0f0 100644
--- a/site/docs/latest/api/ledger-api.md
+++ b/site/docs/latest/api/ledger-api.md
@@ -560,8 +560,7 @@ Available write flags:
 
 | Flag  | Explanation  | Notes |
 :---------|:------------|:-------
-DEFERRED_SYNC | Writes are acknowledged early, without waiting for
-guarantees of durability | Data will be only written to the OS page cache, 
without forcing an fsync.
+DEFERRED_SYNC | Writes are acknowledged early, without waiting for guarantees 
of durability | Data will be only written to the OS page cache, without forcing 
an fsync.
 
 ```java
 BookKeeper bk = ...;
@@ -800,3 +799,43 @@ bk.newDeleteLedgerOp()
     .execute()
     .get();
 ```
+
+### Relaxing Durability
+
+In BookKeeper by default each write will be acklowledged to the client if and 
only if it has been persisted durably (fsync called on the file system) by a 
quorum of bookies.
+In this case the LastAddConfirmed pointer is updated on the writer side, this 
is the guarantee for the writer that data will not be lost and it will
+be always readable by other clients.
+
+On the client side you can temporary relax this constraint by using the 
[`DEFERRED_SYNC`](../javadoc/org/apache/bookkeeper/client/api/WriteFlag) Write 
flag. Using this flag bookies will acknowledge each entry after
+writing the entry to SO buffers without waiting for an fsync.
+In this case the LastAddConfirmed pointer is not advanced to the writer side 
neither is updated on the reader's side, this is because **there is some chance 
to lose the entry**.
+Such entries will be still readable using readUnconfirmed() API, but they 
won't be readable using Long Poll reads or regular read() API.
+
+In order to get guarantees of durability the writer must use explicitly the 
[force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) API 
which will return only after all the bookies in the ensemble ackknowledge the 
call after
+performing an fsync to the disk which is storing the journal.
+This way the LastAddConfirmed pointer is advanced on the writer side and it 
will be eventually available to the readers.
+
+The *close()* operation on the writer writes on ledger's metadata the current 
LastAddConfirmed pointer, **it is up to the application to call force() before 
issuing the close command**.
+In case that you never call explicitly 
[force()](../javadoc/org/apache/bookkeeper/client/api/ForceableHandle) the 
LastAddConfirmed will remain unset (-1) on ledger metadata and regular readers 
won't be able to access data.
+
+
+```java
+BookKeeper bk = ...;
+long ledgerId = ...;
+
+WriteHandle wh = bk.newCreateLedgerOp()
+    .withDigestType(DigestType.CRC32)
+    .withPassword(password)
+    .withEnsembleSize(3)
+    .withWriteQuorumSize(3)
+    .withAckQuorumSize(2)
+    .withWriteFlags(DEFERRED_SYNC)
+    .execute()          // execute the creation op
+    .get();             // wait for the execution to complete
+
+
+wh.force().get();  // wait for fsync, make data available to readers and to 
the replicator
+
+wh.close(); // seal the ledger
+    
+```

Reply via email to