You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@distributedlog.apache.org by si...@apache.org on 2017/04/26 18:41:27 UTC

[03/12] incubator-distributedlog git commit: Release 0.4.0-incubating

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/basic-5.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/basic-5.rst b/website/docs/0.4.0-incubating/tutorials/basic-5.rst
new file mode 100644
index 0000000..94220e8
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/basic-5.rst
@@ -0,0 +1,223 @@
+---
+title: API - Read Records
+top-nav-group: quickstart
+top-nav-pos: 4
+top-nav-title: API - Read Records
+layout: default
+---
+
+.. contents:: Basic Tutorial - Tail reading records from a stream
+
+Tail reading records from a stream
+==================================
+
+This tutorial shows how to tail read records from a stream.
+
+.. sectnum::
+
+Open a distributedlog manager
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Create distributedlog URI
+-------------------------
+
+::
+
+        String dlUriStr = ...;
+        URI uri = URI.create(dlUriStr);
+
+
+Create distributedlog configuration
+-----------------------------------
+
+::
+
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+
+
+Build the distributedlog namespace
+----------------------------------
+
+::
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build(); 
+
+
+Open the distributedlog manager
+-------------------------------
+
+::
+
+        DistributedLogManager dlm = namespace.openLog("basic-stream-9");
+
+
+Get Last Record
+~~~~~~~~~~~~~~~
+
+Get the last record from the record. From the last record, we can use `DLSN` of last record
+to start tailing the stream.
+
+::
+
+        LogRecordWithDLSN record = dlm.getLastLogRecord();
+        DLSN lastDLSN = record.getDlsn();
+
+
+Read Records
+~~~~~~~~~~~~
+
+Open the stream
+---------------
+
+Open the stream to start read the records.
+
+::
+
+        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(lastDLSN));
+
+
+Read the records
+----------------
+
+Read the next available record from the stream. The future is satisified when the record is available.
+
+::
+
+        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+
+
+Register the read callback
+---------------------------
+
+Register a future listener on read completion. The reader will be notified once the record is ready for consuming.
+
+::
+
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                // executed when read failed.
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                // process the record
+                ...
+                // issue read next
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+
+Close the reader
+~~~~~~~~~~~~~~~~
+
+Close the reader after usage.
+
+::
+
+        FutureUtils.result(reader.asyncClose());
+
+
+Run the tutorial
+~~~~~~~~~~~~~~~~
+
+Run the example in the following steps:
+
+Start the local bookkeeper cluster
+----------------------------------
+
+You can use follow command to start the distributedlog stack locally.
+After the distributedlog cluster is started, you could access it using
+distributedlog uri *distributedlog://127.0.0.1:7000/messaging/distributedlog*.
+
+::
+
+        // dlog local ${zk-port}
+        ./distributedlog-core/bin/dlog local 7000
+
+
+Start the write proxy
+---------------------
+
+Start the write proxy, listening on port 8000.
+
+::
+
+        // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+
+
+Create the stream
+-----------------
+
+Create the stream under the distributedlog uri.
+
+::
+
+        // Create Stream `basic-stream-9`
+        // dlog tool create -u ${distributedlog-uri} -r ${stream-prefix} -e ${stream-regex}
+        ./distributedlog-core/bin/dlog tool create -u distributedlog://127.0.0.1:7000/messaging/distributedlog -r basic-stream- -e 9
+
+
+Tail the stream
+---------------
+
+Tailing the stream using `TailReader` to wait for new records.
+
+::
+
+        // Tailing Stream `basic-stream-9`
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-9
+
+
+Write records
+-------------
+
+Run the example to write records to the stream in a console.
+
+::
+
+        // Write Records into Stream `basic-stream-9`
+        // runner run org.apache.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-9
+
+
+Check the results
+-----------------
+
+Example output from `ConsoleProxyWriter` and `TailReader`.
+
+::
+
+        // Output of `ConsoleProxyWriter`
+        May 08, 2016 10:27:41 AM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[inet] = com.twitter.finagle.InetResolver(com.twitter.finagle.InetResolver@756d7bba)
+        May 08, 2016 10:27:41 AM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fixedinet] = com.twitter.finagle.FixedInetResolver(com.twitter.finagle.FixedInetResolver@1d2e91f5)
+        May 08, 2016 10:27:41 AM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[neg] = com.twitter.finagle.NegResolver$(com.twitter.finagle.NegResolver$@5c707aca)
+        May 08, 2016 10:27:41 AM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[nil] = com.twitter.finagle.NilResolver$(com.twitter.finagle.NilResolver$@5c8d932f)
+        May 08, 2016 10:27:41 AM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fail] = com.twitter.finagle.FailResolver$(com.twitter.finagle.FailResolver$@52ba2221)
+        May 08, 2016 10:27:41 AM com.twitter.finagle.Init$$anonfun$1 apply$mcV$sp
+        [dlog] > test-proxy-writer
+        [dlog] >
+
+
+        // Output of `TailReader`
+        Opening log stream basic-stream-9
+        Log stream basic-stream-9 is empty.
+        Wait for records starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        """
+        test-proxy-writer
+        """
+        
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/basic-6.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/basic-6.rst b/website/docs/0.4.0-incubating/tutorials/basic-6.rst
new file mode 100644
index 0000000..4993fe4
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/basic-6.rst
@@ -0,0 +1,327 @@
+---
+title: API - Rewind reading records by time
+layout: default
+---
+
+.. contents:: Basic Tutorial - Rewind reading records by time
+
+Rewind reading records by time
+==============================
+
+This tutorial shows how to rewind reading data from a stream by time.
+
+.. sectnum::
+
+Open a distributedlog manager
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Create distributedlog URI
+-------------------------
+
+::
+
+        String dlUriStr = ...;
+        URI uri = URI.create(dlUriStr);
+
+
+Create distributedlog configuration
+-----------------------------------
+
+::
+
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+
+
+Build the distributedlog namespace
+----------------------------------
+
+::
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build(); 
+
+
+Open the distributedlog manager
+-------------------------------
+
+::
+
+        DistributedLogManager dlm = namespace.openLog("basic-stream-10");
+
+
+Rewind the stream
+~~~~~~~~~~~~~~~~~
+
+Position the reader using timestamp
+-----------------------------------
+
+Since the records written by write proxy will be assigned `System.currentTimeMillis()`
+as the `TransactionID`. It is straightforward to use `TransactionID` to rewind reading
+the records.
+
+::
+
+        int rewindSeconds = 60; // 60 seconds
+        long fromTxID = System.currentTimeMillis() -
+                TimeUnit.MILLISECONDS.convert(rewindSeconds, TimeUnit.SECONDS);
+        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(fromTxID));
+
+
+Read Records
+~~~~~~~~~~~~
+
+Read the next available record from the stream. The future is satisified when the record is available.
+
+::
+
+        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+
+
+Register the read callback
+---------------------------
+
+Register a future listener on read completion. The reader will be notified once the record is ready for consuming.
+
+::
+
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                // executed when read failed.
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                // process the record
+                ...
+                // issue read next
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+
+Close the reader
+~~~~~~~~~~~~~~~~
+
+Close the reader after usage.
+
+::
+
+        FutureUtils.result(reader.asyncClose());
+
+
+Run the tutorial
+~~~~~~~~~~~~~~~~
+
+Run the example in the following steps:
+
+Start the local bookkeeper cluster
+----------------------------------
+
+You can use follow command to start the distributedlog stack locally.
+After the distributedlog cluster is started, you could access it using
+distributedlog uri *distributedlog://127.0.0.1:7000/messaging/distributedlog*.
+
+::
+
+        // dlog local ${zk-port}
+        ./distributedlog-core/bin/dlog local 7000
+
+
+Start the write proxy
+---------------------
+
+Start the write proxy, listening on port 8000.
+
+::
+
+        // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+
+
+Create the stream
+-----------------
+
+Create the stream under the distributedlog uri.
+
+::
+
+        // Create Stream `basic-stream-10`
+        // dlog tool create -u ${distributedlog-uri} -r ${stream-prefix} -e ${stream-regex}
+        ./distributedlog-core/bin/dlog tool create -u distributedlog://127.0.0.1:7000/messaging/distributedlog -r basic-stream- -e 10
+
+
+Generate records
+----------------
+
+Run the `RecordGenerator` to generate records.
+
+::
+
+        // Write Records into Stream `basic-stream-10` in 1 requests/second
+        // runner run org.apache.distributedlog.basic.RecordGenerator ${distributedlog-uri} ${stream} ${rate}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator 'inet!127.0.0.1:8000' basic-stream-10 1
+
+
+Rewind the stream
+-----------------
+
+Rewind the stream using `StreamRewinder` to read records from 30 seconds ago
+
+::
+
+        // Rewind `basic-stream-10`
+        // runner run org.apache.distributedlog.basic.StreamRewinder ${distributedlog-uri} ${stream} ${seconds-to-rewind}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.StreamRewinder distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-10  30
+
+
+Check the results
+-----------------
+
+Example output from `StreamRewinder`.
+
+::
+
+        // Output of `StreamRewinder`
+        Opening log stream basic-stream-10
+        Record records starting from 1462736697481 which is 30 seconds ago
+        Received record DLSN{logSegmentSequenceNo=1, entryId=264, slotId=0}
+        """
+        record-1462736697685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=266, slotId=0}
+        """
+        record-1462736698684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=268, slotId=0}
+        """
+        record-1462736699684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=270, slotId=0}
+        """
+        record-1462736700686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=272, slotId=0}
+        """
+        record-1462736701685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=274, slotId=0}
+        """
+        record-1462736702684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=276, slotId=0}
+        """
+        record-1462736703683
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=278, slotId=0}
+        """
+        record-1462736704685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=280, slotId=0}
+        """
+        record-1462736705686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=282, slotId=0}
+        """
+        record-1462736706682
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=284, slotId=0}
+        """
+        record-1462736707685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=286, slotId=0}
+        """
+        record-1462736708686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=288, slotId=0}
+        """
+        record-1462736709684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=290, slotId=0}
+        """
+        record-1462736710684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=292, slotId=0}
+        """
+        record-1462736711686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=294, slotId=0}
+        """
+        record-1462736712686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=296, slotId=0}
+        """
+        record-1462736713684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=298, slotId=0}
+        """
+        record-1462736714682
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=300, slotId=0}
+        """
+        record-1462736715685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=302, slotId=0}
+        """
+        record-1462736716684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=304, slotId=0}
+        """
+        record-1462736717684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=306, slotId=0}
+        """
+        record-1462736718684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=308, slotId=0}
+        """
+        record-1462736719685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=310, slotId=0}
+        """
+        record-1462736720683
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=312, slotId=0}
+        """
+        record-1462736721686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=314, slotId=0}
+        """
+        record-1462736722685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=316, slotId=0}
+        """
+        record-1462736723683
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=318, slotId=0}
+        """
+        record-1462736724683
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=320, slotId=0}
+        """
+        record-1462736725685
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=322, slotId=0}
+        """
+        record-1462736726686
+        """
+        Reader caught with latest data
+        Received record DLSN{logSegmentSequenceNo=1, entryId=324, slotId=0}
+        """
+        record-1462736727686
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=326, slotId=0}
+        """
+        record-1462736728684
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=328, slotId=0}
+        """
+        record-1462736729682
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=330, slotId=0}
+        """
+        record-1462736730685
+        """

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/main.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/main.rst b/website/docs/0.4.0-incubating/tutorials/main.rst
new file mode 100644
index 0000000..728398e
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/main.rst
@@ -0,0 +1,84 @@
+---
+title: Tutorials
+layout: default
+---
+
+.. contents:: Tutorials
+
+Tutorials
+=========
+
+DistributedLog's documentation shows the architecture, design details,
+available features and how to deploy and manage it. But the documentation
+will not show how to use it. This is where tutorials come in.
+
+Tutorials are useful for showing how DistributedLog is used in real-world
+applications, especially when it comes to integrating with other systems.
+
+Basic
+~~~~~
+
+This section lists the tutorials for basic operations.
+
+- `Write Records using Core Library`_
+
+.. _Write Records using Core Library: {{ site.baseurl }}/tutorials/basic-1
+
+- `Write Records using Write Proxy Client`_
+
+.. _Write Records using Write Proxy Client: {{ site.baseurl }}/tutorials/basic-2
+
+- `Write Records to Multiple Streams`_
+
+.. _Write Records to Multiple Streams: {{ site.baseurl }}/tutorials/basic-3
+
+- `Atomic Write Multiple Records`_
+
+.. _Atomic Write Multiple Records: {{ site.baseurl }}/tutorials/basic-4
+
+- `Tail reading records from a stream`_
+
+.. _Tail reading records from a stream: {{ site.baseurl }}/tutorials/basic-5
+
+- `Rewind reading records by time`_
+
+.. _Rewind reading records by time: {{ site.baseurl }}/tutorials/basic-6
+
+Messaging
+~~~~~~~~~
+
+This section lists the tutorials on how to use `DistributedLog` to build messaging systems.
+
+- `Write records to partitioned streams`_
+
+.. _Write records to partitioned streams: {{ site.baseurl }}/tutorials/messaging-1
+
+- `Write records to multiple streams using a load balancer`_
+
+.. _Write records to multiple streams using a load balancer: {{ site.baseurl }}/tutorials/messaging-2
+
+- `At-least-once Processing`_
+
+.. _At-least-once Processing: {{ site.baseurl }}/tutorials/messaging-3
+
+- `Exact-Once Processing`_
+
+.. _Exact-Once Processing: {{ site.baseurl }}/tutorials/messaging-4
+
+Replicated State Machines
+~~~~~~~~~~~~~~~~~~~~~~~~~
+
+This section lists the tutorials on how to use `DistributedLog` to build reliable distributed systems.
+
+- `Build replicated state machines`_
+
+.. _Build replicated state machines: {{ site.baseurl }}/tutorials/replicatedstatemachines
+
+Analytics
+~~~~~~~~~
+
+This section lists the tutorials on how to use `DistributedLog` for analytics.
+
+- `DistributedLog meets MapReduce`_
+
+.. _DistributedLog meets MapReduce: {{ site.baseurl }}/tutorials/analytics-mapreduce

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/messaging-1.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/messaging-1.rst b/website/docs/0.4.0-incubating/tutorials/messaging-1.rst
new file mode 100644
index 0000000..5536811
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/messaging-1.rst
@@ -0,0 +1,184 @@
+---
+title: Tutorials - Write records to partitioned streams
+top-nav-group: messaging
+top-nav-pos: 1
+top-nav-title: Write records to partitioned streams
+layout: default
+---
+
+.. contents:: Messaging Tutorial - Write records to multiple streams partitioned by key
+
+How to write records to multiple streams partitioning by key
+============================================================
+
+This tutorial shows how to build a multiple-partitioned writer, which writes records to streams partitioned by key.
+
+.. sectnum::
+
+Design a partitioner interface
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In order to implement a multiple-partitioned writer, we need a `Partitioner` to partition the records into different
+streams based on their keys. The partitioner takes a `KEY` and its total partitions to compute a partition id for the
+given `KEY`.
+
+::
+
+    public interface Partitioner<KEY> {
+            int partition(KEY key, int totalPartitions);
+    }
+
+
+Write records based on partition key
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Once we have the `Partitioner` interface, it is easy to implement partitioned writer logic:
+
+- Partitioner takes a `KEY` and the total number of partitions, and compute the partition id for the key.
+- Use the partition id to choose the stream to write.
+- Use `DistributedLogClient.write(stream, ...)` to write the data to the chosen stream.
+
+::
+
+    String[] streams = ...;
+    int pid = partitioner.partition(key, streams.length);
+    ByteBuffer value = ...;
+    client.write(streams[pid], value);
+
+
+
+Run the tutorial
+~~~~~~~~~~~~~~~~
+
+Run the example in the following steps:
+
+Start the local bookkeeper cluster
+----------------------------------
+
+You can use follow command to start the distributedlog stack locally.
+After the distributedlog cluster is started, you could access it using
+distributedlog uri *distributedlog://127.0.0.1:7000/messaging/distributedlog*.
+
+::
+
+        // dlog local ${zk-port}
+        ./distributedlog-core/bin/dlog local 7000
+
+
+Start the write proxy
+---------------------
+
+Start the write proxy, listening on port 8000.
+
+::
+
+        // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+
+
+Create the stream
+-----------------
+
+Create the stream under the distributedlog uri.
+
+::
+
+        // Create Stream `messaging-stream-{1,5}`
+        // dlog tool create -u ${distributedlog-uri} -r ${stream-prefix} -e ${stream-regex}
+        ./distributedlog-core/bin/dlog tool create -u distributedlog://127.0.0.1:7000/messaging/distributedlog -r messaging-stream- -e 1-5
+
+
+Tail the streams
+----------------
+
+Tailing the streams using `MultiReader` to wait for new records.
+
+::
+
+        // Tailing Stream `messaging-stream-{1,5}`
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+
+
+Write records
+-------------
+
+Run the example to write records to multiple stream in a console - the record should be in the form of `KEY:VALUE`.
+
+::
+
+        // Write Records into Stream `messaging-stream-{1,5}`
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+
+
+Check the results
+-----------------
+
+Example output from `ConsoleProxyPartitionedMultiWriter` and `MultiReader`.
+
+::
+
+        // Output of `ConsoleProxyPartitionedMultiWriter`
+        Picked up JAVA_TOOL_OPTIONS: -Dfile.encoding=utf8
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[inet] = com.twitter.finagle.InetResolver(com.twitter.finagle.InetResolver@6c4cbf96)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fixedinet] = com.twitter.finagle.FixedInetResolver(com.twitter.finagle.FixedInetResolver@57052dc3)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[neg] = com.twitter.finagle.NegResolver$(com.twitter.finagle.NegResolver$@14ff89d7)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[nil] = com.twitter.finagle.NilResolver$(com.twitter.finagle.NilResolver$@14b28d06)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fail] = com.twitter.finagle.FailResolver$(com.twitter.finagle.FailResolver$@56488f87)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.Init$$anonfun$1 apply$mcV$sp
+        INFO: Finagle version media-platform-tools/release-20160330-1117-sgerstein-9-g2dcdd6c (rev=2dcdd6c866f9bd3599ed49568d651189735e8ad6) built at 20160330-160058
+        [dlog] > 1:value-1
+        [dlog] > 2:value-2
+        [dlog] > 3:value-3
+        [dlog] > 4:value-4
+        [dlog] > 5:value-5
+        [dlog] >
+
+
+        // Output of `MultiReader`
+        Opening log stream messaging-stream-1
+        Opening log stream messaging-stream-2
+        Opening log stream messaging-stream-3
+        Opening log stream messaging-stream-4
+        Opening log stream messaging-stream-5
+        Log stream messaging-stream-2 is empty.
+        Wait for records from messaging-stream-2 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-2
+        Log stream messaging-stream-1 is empty.
+        Wait for records from messaging-stream-1 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-1
+        Log stream messaging-stream-3 is empty.
+        Wait for records from messaging-stream-3 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-3
+        Log stream messaging-stream-4 is empty.
+        Wait for records from messaging-stream-4 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-4
+        Log stream messaging-stream-5 is empty.
+        Wait for records from messaging-stream-5 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-5
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0} from stream messaging-stream-2
+        """
+        value-1
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0} from stream messaging-stream-3
+        """
+        value-2
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0} from stream messaging-stream-4
+        """
+        value-3
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0} from stream messaging-stream-5
+        """
+        value-4
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0} from stream messaging-stream-1
+        """
+        value-5
+        """

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/messaging-2.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/messaging-2.rst b/website/docs/0.4.0-incubating/tutorials/messaging-2.rst
new file mode 100644
index 0000000..dc1cf5d
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/messaging-2.rst
@@ -0,0 +1,223 @@
+---
+title: Tutorials - Write records to multiple streams using a load balancer
+top-nav-group: messaging
+top-nav-pos: 2
+top-nav-title: Write records to multiple streams (load balancer)
+layout: default
+---
+
+.. contents:: Messaging Tutorial - Write records to multiple streams using a load balancer
+
+How to write records to multiple streams using a load balancer
+==============================================================
+
+If applications does not care about ordering and they just want use multiple streams to transform messages, it is easier
+to use a load balancer to balancing the writes among the multiple streams.
+
+This tutorial shows how to build a multi-streams writer, which ues a finagle load balancer to balancing traffic among multiple streams.
+
+.. sectnum::
+
+Make writing to a stream as a finagle service
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In order to leverage the finagle load balancer to balancing traffic among multiple streams, we have to make writing
+to a stream as a finagle service.
+
+::
+
+    class StreamWriter<VALUE> extends Service<VALUE, DLSN> {
+
+        private final String stream;
+        private final DistributedLogClient client;
+
+        StreamWriter(String stream,
+                     DistributedLogClient client) {
+            this.stream = stream;
+            this.client = client;
+        }
+
+        @Override
+        public Future<DLSN> apply(VALUE request) {
+            return client.write(stream, ByteBuffer.wrap(request.toString().getBytes(UTF_8)));
+        }
+    } 
+
+
+Create a load balancer from multiple streams
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Create a ServiceFactory
+-----------------------
+
+Create a set of finagle `ServiceFactory` over multiple streams.
+
+::
+
+        String[] streams;
+        Set<ServiceFactory<VALUE, DLSN>> serviceFactories = Sets.newHashSet();
+        for (String stream : streams) {
+            Service<VALUE, DLSN> service = new StreamWriter(stream, client);
+            serviceFactories.add(new SingletonFactory<VALUE, DLSN>(service));
+        }
+
+
+Create the load balancer
+------------------------
+
+::
+
+        Service<VALUE, DLSN> writeSerivce =
+            Balancers.heap(new scala.util.Random(System.currentTimeMillis()))
+                .newBalancer(
+                        Activity.value(scalaSet),
+                        NullStatsReceiver.get(),
+                        new NoBrokersAvailableException("No partitions available")
+                ).toService();
+
+
+Write records
+~~~~~~~~~~~~~
+
+Once the balancer service is initialized, we can write records through the balancer service.
+
+::
+
+    Future<DLSN> writeFuture = writeSerivce.write(...);
+
+
+Run the tutorial
+~~~~~~~~~~~~~~~~
+
+Run the example in the following steps:
+
+Start the local bookkeeper cluster
+----------------------------------
+
+You can use follow command to start the distributedlog stack locally.
+After the distributedlog cluster is started, you could access it using
+distributedlog uri *distributedlog://127.0.0.1:7000/messaging/distributedlog*.
+
+::
+
+        // dlog local ${zk-port}
+        ./distributedlog-core/bin/dlog local 7000
+
+
+Start the write proxy
+---------------------
+
+Start the write proxy, listening on port 8000.
+
+::
+
+        // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+
+
+Create the stream
+-----------------
+
+Create the stream under the distributedlog uri.
+
+::
+
+        // Create Stream `messaging-stream-{1,5}`
+        // dlog tool create -u ${distributedlog-uri} -r ${stream-prefix} -e ${stream-regex}
+        ./distributedlog-core/bin/dlog tool create -u distributedlog://127.0.0.1:7000/messaging/distributedlog -r messaging-stream- -e 1-5
+
+
+Tail the streams
+----------------
+
+Tailing the streams using `MultiReader` to wait for new records.
+
+::
+
+        // Tailing Stream `messaging-stream-{1,5}`
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+
+
+Write records
+-------------
+
+Run the example to write records to multiple stream in a console.
+
+::
+
+        // Write Records into Stream `messaging-stream-{1,5}`
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+
+
+Check the results
+-----------------
+
+Example output from `ConsoleProxyRRMultiWriter` and `MultiReader`.
+
+::
+
+        // Output of `ConsoleProxyRRMultiWriter`
+        Picked up JAVA_TOOL_OPTIONS: -Dfile.encoding=utf8
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[inet] = com.twitter.finagle.InetResolver(com.twitter.finagle.InetResolver@6c4cbf96)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fixedinet] = com.twitter.finagle.FixedInetResolver(com.twitter.finagle.FixedInetResolver@57052dc3)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[neg] = com.twitter.finagle.NegResolver$(com.twitter.finagle.NegResolver$@14ff89d7)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[nil] = com.twitter.finagle.NilResolver$(com.twitter.finagle.NilResolver$@14b28d06)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.BaseResolver$$anonfun$resolvers$1 apply
+        INFO: Resolver[fail] = com.twitter.finagle.FailResolver$(com.twitter.finagle.FailResolver$@56488f87)
+        May 08, 2016 1:22:35 PM com.twitter.finagle.Init$$anonfun$1 apply$mcV$sp
+        INFO: Finagle version media-platform-tools/release-20160330-1117-sgerstein-9-g2dcdd6c (rev=2dcdd6c866f9bd3599ed49568d651189735e8ad6) built at 20160330-160058
+        [dlog] > message-1
+        [dlog] > message-2
+        [dlog] > message-3
+        [dlog] > message-4
+        [dlog] > message-5
+        [dlog] >
+
+
+        // Output of `MultiReader`
+        Opening log stream messaging-stream-1
+        Opening log stream messaging-stream-2
+        Opening log stream messaging-stream-3
+        Opening log stream messaging-stream-4
+        Opening log stream messaging-stream-5
+        Log stream messaging-stream-2 is empty.
+        Wait for records from messaging-stream-2 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-2
+        Log stream messaging-stream-1 is empty.
+        Wait for records from messaging-stream-1 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-1
+        Log stream messaging-stream-3 is empty.
+        Wait for records from messaging-stream-3 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-3
+        Log stream messaging-stream-4 is empty.
+        Wait for records from messaging-stream-4 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-4
+        Log stream messaging-stream-5 is empty.
+        Wait for records from messaging-stream-5 starting from DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}
+        Open reader to read records from stream messaging-stream-5
+        Received record DLSN{logSegmentSequenceNo=1, entryId=2, slotId=0} from stream messaging-stream-3
+        """
+        message-1
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=2, slotId=0} from stream messaging-stream-2
+        """
+        message-2
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=4, slotId=0} from stream messaging-stream-2
+        """
+        message-3
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=2, slotId=0} from stream messaging-stream-4
+        """
+        message-4
+        """
+        Received record DLSN{logSegmentSequenceNo=1, entryId=6, slotId=0} from stream messaging-stream-2
+        """
+        message-5
+        """

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/messaging-3.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/messaging-3.rst b/website/docs/0.4.0-incubating/tutorials/messaging-3.rst
new file mode 100644
index 0000000..b7618e2
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/messaging-3.rst
@@ -0,0 +1,111 @@
+---
+title: Tutorials - At-least-once Processing
+top-nav-group: messaging
+top-nav-pos: 3
+top-nav-title: At-least-once Processing
+layout: default
+---
+
+.. contents:: Messaging Tutorial - At-least-once Processing
+
+At-least-once Processing
+========================
+
+Applications typically choose between `at-least-once` and `exactly-once` processing semantics.
+`At-least-once` processing guarantees that the application will process all the log records,
+however when the application resumes after failure, previously processed records may be re-processed
+if they have not been acknowledged. With at least once processing guarantees the application can store
+reader positions in an external store and update it periodically. Upon restart the application will
+reprocess messages since the last updated reader position.
+
+This tutorial shows how to do `at-least-once` processing by using a `offset-store` to track the reading positions.
+
+.. sectnum::
+
+How to track reading positions
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Applications typically choose an external storage (e.g key/value storage) or another log stream to store their
+read positions. In this example, we used a local key/value store - `LevelDB` to store the read positions.
+
+Open the offset store
+---------------------
+
+::
+
+        String offsetStoreFile = ...;
+        Options options = new Options();
+        options.createIfMissing(true);
+        DB offsetDB = factory.open(new File(offsetStoreFile), options);
+
+
+Read the reader read position
+-----------------------------
+
+Read the reader read position from the offset store.
+
+::
+
+        byte[] offset = offsetDB.get(readerId.getBytes(UTF_8));
+        DLSN dlsn;
+        if (null == offset) {
+            dlsn = DLSN.InitialDLSN;
+        } else {
+            dlsn = DLSN.deserializeBytes(offset);
+        }
+
+
+Read records
+------------
+
+Start read from the read position that recorded in offset store.
+
+::
+
+        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(dlsn));
+
+
+Track read position
+-------------------
+
+Track the last read position while reading using `AtomicReference`.
+
+::
+
+        final AtomicReference<DLSN> lastReadDLSN = new AtomicReference<DLSN>(null);
+        reader.readNext().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+            ...
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                lastReadDLSN.set(record.getDlsn()); 
+                // process the record
+                ...
+                // read next record
+                reader.readNext().addEventListener(this);
+            }
+        });
+
+
+Record read position
+--------------------
+
+Periodically update the last read position to the offset store.
+
+::
+
+        final ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        executorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (null != lastDLSN.get()) {
+                    offsetDB.put(readerId.getBytes(UTF_8), lastDLSN.get().serializeBytes());
+                }
+            }
+        }, 10, 10, TimeUnit.SECONDS);
+
+
+Check `distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets`_ for more details.
+
+.. _distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets: https://github.com/apache/incubator-distributedlog/blob/master/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets.java

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/messaging-4.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/messaging-4.rst b/website/docs/0.4.0-incubating/tutorials/messaging-4.rst
new file mode 100644
index 0000000..7e585b2
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/messaging-4.rst
@@ -0,0 +1,24 @@
+---
+title: Tutorials - Exact-Once Processing
+top-nav-group: messaging
+top-nav-pos: 4
+top-nav-title: Exact-Once Processing
+layout: default
+---
+
+.. contents:: Messaging Tutorial - Exact-Once Processing
+
+Exact-Once Processing
+=====================
+
+Applications typically choose between `at-least-once` and `exactly-once` processing semantics.
+`At-least-once` processing guarantees that the application will process all the log records,
+however when the application resumes after failure, previously processed records may be re-processed
+if they have not been acknowledged. `Exactly once` processing is a stricter guarantee where applications
+must see the effect of processing each record exactly once. `Exactly once` semantics can be achieved
+by maintaining reader positions together with the application state and atomically updating both the
+reader position and the effects of the corresponding log records. 
+
+This tutorial_ shows how to do `exact-once` processing.
+
+.. _tutorial: https://github.com/apache/incubator-distributedlog/blob/master/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/StreamTransformer.java

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/messaging-5.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/messaging-5.rst b/website/docs/0.4.0-incubating/tutorials/messaging-5.rst
new file mode 100644
index 0000000..12f4181
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/messaging-5.rst
@@ -0,0 +1,19 @@
+---
+title: Tutorials - Implement a kafka-like pub/sub system
+top-nav-group: messaging
+top-nav-pos: 5
+top-nav-title: Implement a kafka-like pub/sub system
+layout: default
+---
+
+.. contents:: Messaging Tutorial - Implement a kafka-like pub/sub system
+
+How to implement a kafka-like partitioned pub/sub system using DistributedLog
+=============================================================================
+
+`./distributedlog-tutorials/distributedlog-kafka`_ shows how to implement a kafka-like publisher using multiple streams.
+Check `KafkaDistributedLogProducer`_ for more details.
+
+.. _./distributedlog-tutorials/distributedlog-kafka: https://github.com/apache/incubator-distributedlog/tree/master/distributedlog-tutorials/distributedlog-kafka
+.. _KafkaDistributedLogProducer: https://github.com/apache/incubator-distributedlog/blob/master/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/KafkaDistributedLogProducer.java
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/tutorials/replicatedstatemachines.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/tutorials/replicatedstatemachines.rst b/website/docs/0.4.0-incubating/tutorials/replicatedstatemachines.rst
new file mode 100644
index 0000000..dc5e8e9
--- /dev/null
+++ b/website/docs/0.4.0-incubating/tutorials/replicatedstatemachines.rst
@@ -0,0 +1,11 @@
+---
+title: Tutorial - Build replicated state machines
+top-nav-group: replicatedstatemachine
+top-nav-pos: 1
+top-nav-title: Build replicated state machines 
+layout: default
+---
+
+.. contents:: Tutorial - Build replicated state machines
+
+[Coming Soon]

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/user_guide/api/core.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/user_guide/api/core.rst b/website/docs/0.4.0-incubating/user_guide/api/core.rst
new file mode 100644
index 0000000..7930e00
--- /dev/null
+++ b/website/docs/0.4.0-incubating/user_guide/api/core.rst
@@ -0,0 +1,485 @@
+---
+title: "Core Library"
+layout: default
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: apis
+sub-nav-pos: 3
+sub-nav-title: Core Library API
+
+---
+
+.. contents:: Core Library API
+
+Core Library API
+================
+
+The distributedlog core library interacts with namespaces and logs directly.
+It is written in Java.
+
+Namespace API
+-------------
+
+A DL namespace is a collection of *log streams*. Applications could *create*
+or *delete* logs under a DL namespace.
+
+Namespace URI
+~~~~~~~~~~~~~
+
+An **URI** is used to locate the *namespace*. The *Namespace URI* is typically
+comprised of *3* components:
+
+* scheme: `distributedlog-<backend>`. The *backend* indicates what backend is used to store the log data.
+* domain name: the domain name that used to talk to the *backend*. In the example as below, the domain name part is *zookeeper server*, which is used to store log metadata in bookkeeper based backend implementation.
+* path: path points to the location that stores logs. In the example as below, it is a zookeeper path that points to the znode that stores all the logs metadata.
+
+::
+
+    distributedlog-bk://<zookeeper-server>/path/to/stream
+
+The available backend is only bookkeeper based backend.
+The default `distributedlog` scheme is aliased to `distributedlog-bk`.
+
+Building a Namespace
+~~~~~~~~~~~~~~~~~~~~
+
+Once you have the *namespace uri*, you could build the namespace instance.
+The namespace instance will be used for operating streams under it.
+
+::
+
+    // DistributedLog Configuration
+    DistributedLogConfiguration conf = new DistributedLogConfiguration();
+    // Namespace URI
+    URI uri = ...; // create the namespace uri
+    // create a builder to build namespace instances
+    DistributedLogNamespaceBuilder builder = DistributedLogNamespaceBuilder.newBuilder();
+    DistributedLogNamespace namespace = builder
+        .conf(conf)             // configuration that used by namespace
+        .uri(uri)               // namespace uri
+        .statsLogger(...)       // stats logger to log stats
+        .featureProvider(...)   // feature provider on controlling features
+        .build();
+
+Create a Log
+~~~~~~~~~~~~
+
+Creating a log is pretty straight forward by calling `distributedlognamespace#createlog(logname)`.
+it only creates the log under the namespace but doesn't return any handle for operating the log.
+
+::
+
+    DistributedLogNamespace namespace = ...; // namespace
+    try {
+        namespace.createLog("test-log");
+    } catch (IOException ioe) {
+        // handling the exception on creating a log
+    }
+
+Open a Log
+~~~~~~~~~~
+
+A `DistributedLogManager` handle will be returned when opening a log by `#openLog(logName)`. The
+handle could be used for writing data to or reading data from the log. If the log doesn't exist
+and `createStreamIfNotExists` is set to true in the configuration, the log will be created
+automatically when writing first record.
+
+::
+
+    DistributedLogConfiguration conf = new DistributedLogConfiguration();
+    conf.setCreateStreamIfNotExists(true);
+    DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder()
+        .conf(conf)
+        ...
+        .build();
+    DistributedLogManager logManager = namespace.openLog("test-log");
+    // use the log manager to open writer to write data or open reader to read data
+    ...
+
+Sometimes, applications may open a log with different configuration settings. It could be done via
+a overloaded `#openLog` method, as below:
+
+::
+
+    DistributedLogConfiguration conf = new DistributedLogConfiguration();
+    // set the retention period hours to 24 hours.
+    conf.setRetentionPeriodHours(24);
+    URI uri = ...;
+    DistributedLogNamespace namespace = DistributedLogNamespace.newBuilder()
+        .conf(conf)
+        .uri(uri)
+        ...
+        .build();
+
+    // Per Log Configuration
+    DistributedLogConfigration logConf = new DistributedLogConfiguration();
+    // set the retention period hours to 12 hours for a single stream
+    logConf.setRetentionPeriodHours(12);
+
+    // open the log with overrided settings
+    DistributedLogManager logManager = namespace.openLog("test-log",
+        Optional.of(logConf),
+        Optiona.absent());
+
+Delete a Log
+~~~~~~~~~~~~
+
+`DistributedLogNamespace#deleteLog(logName)` will deletes the log from the namespace. Deleting a log
+will attempt acquiring a lock before deletion. If a log is writing by an active writer, the lock
+would be already acquired by the writer. so the deleting will fail.
+
+::
+
+    DistributedLogNamespace namespace = ...;
+    try {
+        namespace.deleteLog("test-log");
+    } catch (IOException ioe) {
+        // handle the exceptions
+    }
+
+Log Existence
+~~~~~~~~~~~~~
+
+Applications could check whether a log exists in a namespace by calling `DistributedLogNamespace#logExists(logName)`.
+
+::
+
+    DistributedLogNamespace namespace = ...;
+    if (namespace.logExists("test-log")) {
+        // actions when log exists
+    } else {
+        // actions when log doesn't exist
+    }
+
+Get List of Logs
+~~~~~~~~~~~~~~~~
+
+Applications could list the logs under a namespace by calling `DistributedLogNamespace#getLogs()`.
+
+::
+
+    DistributedLogNamespace namespace = ...;
+    Iterator<String> logs = namespace.getLogs();
+    while (logs.hasNext()) {
+        String logName = logs.next();
+        // ... process the log
+    }
+
+Writer API
+----------
+
+There are two ways to write records into a log stream, one is using 'synchronous' `LogWriter`, while the other one is using
+asynchronous `AsyncLogWriter`.
+
+LogWriter
+~~~~~~~~~
+
+The first thing to write data into a log stream is to construct the writer instance. Please note that the distributedlog core library enforce single-writer
+semantic by deploying a zookeeper locking mechanism. If there is only an active writer, the subsequent calls to `#startLogSegmentNonPartitioned()` will
+fail with `OwnershipAcquireFailedException`.
+
+::
+    
+    DistributedLogNamespace namespace = ....;
+    DistributedLogManager dlm = namespace.openLog("test-log");
+    LogWriter writer = dlm.startLogSegmentNonPartitioned();
+
+.. _Construct Log Record:
+
+Log records are constructed to represent the data written to a log stream. Each log record is associated with application defined transaction id.
+The transaction id has to be non-decreasing otherwise writing the record will be rejected with `TransactionIdOutOfOrderException`. Application is allowed to
+bypass the transaction id sanity checking by setting `maxIdSanityCheck` to false in configuration. System time and atomic numbers are good candicates used for
+transaction id.
+
+::
+
+    long txid = 1L;
+    byte[] data = ...;
+    LogRecord record = new LogRecord(txid, data);
+
+Application could either add a single record (via `#write(LogRecord)`) or a bunch of records (via `#writeBulk(List<LogRecord>)`) into the log stream.
+
+::
+
+    writer.write(record);
+    // or
+    List<LogRecord> records = Lists.newArrayList();
+    records.add(record);
+    writer.writeBulk(records);
+
+The write calls return immediately after the records are added into the output buffer of writer. So the data isn't guaranteed to be durable until writer
+explicitly calls `#setReadyToFlush()` and `#flushAndSync()`. Those two calls will first transmit buffered data to backend, wait for transmit acknowledges
+and commit the written data to make them visible to readers.
+
+::
+
+    // flush the records
+    writer.setReadyToFlush();
+    // commit the records to make them visible to readers
+    writer.flushAndSync();
+
+The DL log streams are endless streams unless they are sealed. 'endless' means that writers keep writing records to the log streams, readers could keep
+tailing reading from the end of the streams and it never stops. Application could seal a log stream by calling `#markEndOfStream()`.
+
+::
+
+    // seal the log stream
+    writer.markEndOfStream();
+    
+
+The complete example of writing records is showed as below.
+
+::
+
+    DistributedLogNamespace namespace = ....;
+    DistributedLogManager dlm = namespace.openLog("test-log");
+
+    LogWriter writer = dlm.startLogSegmentNonPartitioned();
+    for (long txid = 1L; txid <= 100L; txid++) {
+        byte[] data = ...;
+        LogRecord record = new LogRecord(txid, data);
+        writer.write(record);
+    }
+    // flush the records
+    writer.setReadyToFlush();
+    // commit the records to make them visible to readers
+    writer.flushAndSync();
+
+    // seal the log stream
+    writer.markEndOfStream();
+
+AsyncLogWriter
+~~~~~~~~~~~~~~
+
+Constructing an asynchronous `AsyncLogWriter` is as simple as synchronous `LogWriter`.
+
+::
+
+    DistributedLogNamespace namespace = ....;
+    DistributedLogManager dlm = namespace.openLog("test-log");
+    AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+
+All the writes to `AsyncLogWriter` are asynchronous. The futures representing write results are only satisfied when the data are persisted in the stream durably.
+A DLSN (distributedlog sequence number) will be returned for each write, which is used to represent the position (aka offset) of the record in the log stream.
+All the records adding in order are guaranteed to be persisted in order.
+
+.. _Async Write Records:
+
+::
+
+    List<Future<DLSN>> addFutures = Lists.newArrayList();
+    for (long txid = 1L; txid <= 100L; txid++) {
+        byte[] data = ...;
+        LogRecord record = new LogRecord(txid, data);
+        addFutures.add(writer.write(record));
+    }
+    List<DLSN> addResults = Await.result(Future.collect(addFutures));
+
+The `AsyncLogWriter` also provides the method to truncate a stream to a given DLSN. This is super helpful for building replicated state machines, who need
+explicit controls on when the data could be deleted.
+
+::
+    
+    DLSN truncateDLSN = ...;
+    Future<DLSN> truncateFuture = writer.truncate(truncateDLSN);
+    // wait for truncation result
+    Await.result(truncateFuture);
+
+Reader API
+----------
+
+Sequence Numbers
+~~~~~~~~~~~~~~~~
+
+A log record is associated with sequence numbers. First of all, application can assign its own sequence number (called `TransactionID`)
+to the log record while writing it (see `Construct Log Record`_). Secondly, a log record will be assigned with an unique system generated sequence number
+`DLSN` (distributedlog sequence number) when it is written to a log (see `Async Write Records`_). Besides `DLSN` and `TransactionID`,
+a monotonically increasing 64-bits `SequenceId` is assigned to the record at read time, indicating its position within the log.
+
+:Transaction ID: Transaction ID is a positive 64-bits long number that is assigned by the application.
+    Transaction ID is very helpful when application wants to organize the records and position the readers using their own sequencing method. A typical
+    use case of `Transaction ID` is `DistributedLog Write Proxy`. The write proxy assigns non-decreasing timestamps to log records, which the timestamps
+    could be used as `physical time` to implement `TTL` (Time To Live) feature in a strong consistent database.
+:DLSN: DLSN (DistributedLog Sequence Number) is the sequence number generated during written time.
+    DLSN is comparable and could be used to figure out the order between records. A DLSN is comprised with 3 components. They are `Log Segment Sequence Number`,
+    `Entry Id` and `Slot Id`. The DLSN is usually used for comparison, positioning and truncation.
+:Sequence ID: Sequence ID is introduced to address the drawback of `DLSN`, in favor of answering questions like `how many records written between two DLSNs`.
+    Sequence ID is a 64-bits monotonic increasing number starting from zero. The sequence ids are computed during reading, and only accessible by readers.
+    That means writers don't know the sequence ids of records at the point they wrote them.
+
+The readers could be positioned to start reading from any positions in the log, by using `DLSN` or `Transaction ID`.
+
+LogReader
+~~~~~~~~~
+
+`LogReader` is a 'synchronous' sequential reader reading records from a log stream starting from a given position. The position could be
+`DLSN` (via `#getInputStream(DLSN)`) or `Transaction ID` (via `#getInputStream(long)`). After the reader is open, it could call either
+`#readNext(boolean)` or `#readBulk(boolean, int)` to read records out of the log stream sequentially. Closing the reader (via `#close()`)
+will release all the resources occupied by this reader instance.
+
+Exceptions could be thrown during reading records due to various issues. Once the exception is thrown, the reader is set to an error state
+and it isn't usable anymore. It is the application's responsibility to handle the exceptions and re-create readers if necessary.
+
+::
+    
+    DistributedLogManager dlm = ...;
+    long nextTxId = ...;
+    LogReader reader = dlm.getInputStream(nextTxId);
+
+    while (true) { // keep reading & processing records
+        LogRecord record;
+        try {
+            record = reader.readNext(false);
+            nextTxId = record.getTransactionId();
+            // process the record
+            ...
+        } catch (IOException ioe) {
+            // handle the exception
+            ...
+            reader = dlm.getInputStream(nextTxId + 1);
+        }
+    }
+
+Reading records from an endless log stream in `synchronous` way isn't as trivial as in `asynchronous` way. Because it lacks of callback mechanism.
+Instead, `LogReader` introduces a flag `nonBlocking` on controlling the waiting behavior on `synchronous` reads. Blocking (`nonBlocking = false`)
+means the reads will wait for records before returning read calls, while NonBlocking (`nonBlocking = true`) means the reads will only check readahead
+cache and return whatever records available in readahead cache.
+
+The `waiting` period varies in `blocking` mode. If the reader is catching up with writer (there are plenty of records in the log), the read call will
+wait until records are read and returned. If the reader is already caught up with writer (there are no more records in the log at read time), the read
+call will wait for a small period of time (defined in `DistributedLogConfiguration#getReadAheadWaitTime()`) and return whatever records available in
+readahead cache. In other words, if a reader sees no record on blocking reads, it means the reader is `caught-up` with the writer.
+
+See examples below on how to read records using `LogReader`.
+
+::
+
+    // Read individual records
+    
+    LogReader reader = ...;
+    // keep reading records in blocking mode until no records available in the log
+    LogRecord record = reader.readNext(false);
+    while (null != record) {
+        // process the record
+        ...
+        // read next record
+        record = reader.readNext(false);
+    }
+    ...
+
+    // reader is caught up with writer, doing non-blocking reads to tail the log
+    while (true) {
+        record = reader.readNext(true);
+        if (null == record) {
+            // no record available yet. backoff ?
+            ...
+        } else {
+            // process the new record
+            ...
+        }
+    }
+
+::
+    
+    // Read records in batch
+
+    LogReader reader = ...;
+    int N = 10;
+
+    // keep reading N records in blocking mode until no records available in the log
+    List<LogRecord> records = reader.readBulk(false, N);
+    while (!records.isEmpty()) {
+        // process the list of records
+        ...
+        if (records.size() < N) { // no more records available in the log
+            break;
+        }
+        // read next N records
+        records = reader.readBulk(false, N);
+    }
+
+    ...
+
+    // reader is caught up with writer, doing non-blocking reads to tail the log
+    while (true) {
+        records = reader.readBulk(true, N);
+        // process the new records
+        ...
+    }
+
+
+AsyncLogReader
+~~~~~~~~~~~~~~
+
+Similar as `LogReader`, applications could open an `AsyncLogReader` by positioning to different positions, either `DLSN` or `Transaction ID`.
+
+::
+    
+    DistributedLogManager dlm = ...;
+
+    Future<AsyncLogReader> openFuture;
+
+    // position the reader to transaction id `999`
+    openFuture = dlm.openAsyncLogReader(999L);
+
+    // or position the reader to DLSN
+    DLSN fromDLSN = ...;
+    openFuture = dlm.openAsyncLogReader(fromDLSN);
+
+    AsyncLogReader reader = Await.result(openFuture);
+
+Reading records from an `AsyncLogReader` is asynchronously. The future returned by `#readNext()`, `#readBulk(int)` or `#readBulk(int, long, TimeUnit)`
+represents the result of the read operation. The future is only satisfied when there are records available. Application could chain the futures
+to do sequential reads.
+
+Reading records one by one from an `AsyncLogReader`.
+
+::
+
+    void readOneRecord(AsyncLogReader reader) {
+        reader.readNext().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+            public void onSuccess(LogRecordWithDLSN record) {
+                // process the record
+                ...
+                // read next
+                readOneRecord(reader);
+            }
+            public void onFailure(Throwable cause) {
+                // handle errors and re-create reader
+                ...
+                reader = ...;
+                // read next
+                readOneRecord(reader);
+            }
+        });
+    }
+    
+    AsyncLogReader reader = ...;
+    readOneRecord(reader);
+
+Reading records in batches from an `AsyncLogReader`.
+
+::
+
+    void readBulk(AsyncLogReader reader, int N) {
+        reader.readBulk(N).addEventListener(new FutureEventListener<List<LogRecordWithDLSN>>() {
+            public void onSuccess(List<LogRecordWithDLSN> records) {
+                // process the records
+                ...
+                // read next
+                readBulk(reader, N);
+            }
+            public void onFailure(Throwable cause) {
+                // handle errors and re-create reader
+                ...
+                reader = ...;
+                // read next
+                readBulk(reader, N);
+            }
+        });
+    }
+    
+    AsyncLogReader reader = ...;
+    readBulk(reader, N);
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/user_guide/api/main.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/user_guide/api/main.rst b/website/docs/0.4.0-incubating/user_guide/api/main.rst
new file mode 100644
index 0000000..d344d4f
--- /dev/null
+++ b/website/docs/0.4.0-incubating/user_guide/api/main.rst
@@ -0,0 +1,37 @@
+---
+title: "Programming Guide"
+layout: default
+
+# Top navigation
+top-nav-group: user-guide
+top-nav-pos: 4
+top-nav-title: API
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: user-guide
+sub-nav-id: apis
+sub-nav-pos: 4
+sub-nav-title: API
+---
+
+API
+===
+
+This page covers the API usage of `DistributedLog`.
+
+- `Core Library API`_
+
+.. _Core Library API: ./core
+
+- `Proxy Client API`_
+
+.. _Proxy Client API: ./proxy
+
+- `Best Practise`_
+
+.. _Best Practise: ./practice
+
+See the `Java API Reference`_ for more information on individual APIs.
+
+.. _Java API Reference: ../../api/java

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/user_guide/api/practice.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/user_guide/api/practice.rst b/website/docs/0.4.0-incubating/user_guide/api/practice.rst
new file mode 100644
index 0000000..01bf9ac
--- /dev/null
+++ b/website/docs/0.4.0-incubating/user_guide/api/practice.rst
@@ -0,0 +1,89 @@
+---
+layout: default
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: apis
+sub-nav-pos: 5
+sub-nav-title: Best Practise
+
+---
+
+.. contents:: Best Practices
+
+Best Practices
+==============
+
+Write records using Fat Client or Thin Client
+---------------------------------------------
+
+`Fat Client` is the writer in distributedlog core library which talks to ZooKeeper and BookKeeper directly,
+while `Thin Client` is the write proxy client which talks to write proxy service.
+
+It is strongly recommended that writing records via `Write Proxy` service rather than using core library directly.
+Because using `Thin Client` has following benefits:
+
+- `Thin Client` is purely thrift RPC based client. It doesn't talk to zookeeper and bookkeeper directly and less complicated.
+- `Write Proxy` manages ownerships of log writers. `Thin Client` doesn't have to deal with ownerships of log writers.
+- `Thin Client` is more upgrade-friendly than `Fat Client`.
+
+The only exception to use distributedlog core library directly is when you application requires:
+
+- Write Ordering. `Write Ordering` means all the writes issued by the writer should be written in a strict order
+  in the log. `Write Proxy` service could only guarantee `Read Ordering`. `Read Ordering` means the write proxies will write 
+  the write requests in their receiving order and gurantee the data seen by all the readers in same order.
+- Ownership Management. If the application already has any kind of ownership management, like `master-slave`, it makes more
+  sense that it uses distributedlog core library directly.
+
+How to position reader by time
+------------------------------
+
+Sometimes, application wants to read data by time, like read data from 2 hours ago. This could be done by positioning
+the reader using `Transaction ID`, if the `Transaction ID` is the timestamp (All the streams produced by `Write Proxy` use
+timestamp as `Transaction ID`).
+
+::
+
+    DistributedLogManager dlm = ...;
+
+    long timestamp = System.currentTimeMillis();
+    long startTxId = timestamp - TimeUnit.MILLISECONDS.convert(2, TimeUnit.HOURS);
+    AsyncLogReader reader = Await.result(dlm.openAsyncLogReader(startTxId));
+    ...
+
+How to seal a stream
+--------------------
+
+Typically, DistributedLog is used as endless streams. In some use cases, application wants to `seal` the stream. So writers
+can't write more data into the log stream and readers could receive notifications about the stream has been sealed.
+
+Write could seal a log stream as below:
+
+::
+    
+    DistributedLogManager dlm = ...;
+
+    LogWriter writer = dlm.startLogSegmentNonPartitioned;
+    // writer writes bunch of records
+    ...
+
+    // writer seals the stream
+    writer.markEndOfStream();
+
+Reader could detect a stream has been sealed as below:
+
+::
+    
+    DistributedLogManager dlm = ...;
+
+    LogReader reader = dlm.getInputStream(1L);
+    LogRecord record;
+    try {
+        while ((record = reader.readNext(false)) != null) {
+            // process the record
+            ...
+        }
+    } catch (EndOfStreamException eos) {
+        // the stream has been sealed
+        ...
+    }

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/user_guide/api/proxy.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/user_guide/api/proxy.rst b/website/docs/0.4.0-incubating/user_guide/api/proxy.rst
new file mode 100644
index 0000000..680d8bc
--- /dev/null
+++ b/website/docs/0.4.0-incubating/user_guide/api/proxy.rst
@@ -0,0 +1,87 @@
+---
+layout: default
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: apis
+sub-nav-pos: 4
+sub-nav-title: Proxy Client API
+
+---
+
+.. contents:: Write Proxy Client API
+
+Write Proxy Client API
+======================
+
+`Write Proxy` is a 'stateless' service on managing the ownerships of writers of log streams. It is used to
+accept to `fan-in` writes from different publishers.
+
+Build Client
+------------
+ 
+The first thing of using `Write Proxy` service is to build the write proxy client. The endpoint of a `Write Proxy` service
+is typically identified by `Finagle Name`_. Name strings must be supplied when constructing a `Write Proxy` client.
+
+.. _Finagle Name: http://twitter.github.io/finagle/guide/Names.html
+
+::
+
+    // 1. Create a Finagle client builder. It would be used for building connection to write proxies.
+    ClientBuilder clientBuilder = ClientBuilder.get()
+        .hostConnectionLimit(1)
+        .hostConnectionCoresize(1)
+        .tcpConnectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+        .connectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+        .requestTimeout(Duration$.MODULE$.fromSeconds(2));
+
+    // 2. Choose a client id to identify the client.
+    ClientId clientId = ClientId$.MODULE$.apply("test");
+
+    String finagleNameStr = "inet!127.0.0.1:8000";
+    
+    // 3. Create the write proxy client builder
+    DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+        .name("test-writer")
+        .clientId(clientId)
+        .clientBuilder(clientBuilder)
+        .statsReceiver(statsReceiver)
+        .finagleNameStr(finagleNameStr);
+
+    // 4. Build the client
+    DistributedLogClient client = builder.build();
+
+Write Records
+-------------
+
+Writing records to log streams via `Write Proxy` is much simpler than using the core library. The transaction id
+will be automatically assigned with `timestamp` by write proxies. The `timestamp` is guaranteed to be non-decreasing, which it
+could be treated as `physical time` within a log stream, and be used for implementing features like `TTL` in a strong consistent
+database.
+
+::
+    
+    DistributedLogClient client = ...;
+
+    // Write a record to a stream
+    String streamName = "test-stream";
+    byte[] data = ...;
+    Future<DLSN> writeFuture = client.write(streamName, ByteBuffer.wrap(data));
+    Await.result(writeFuture);
+
+Truncate Streams
+----------------
+
+Client could issue truncation requests (via `#truncate(String, DLSN)`) to write proxies to truncate a log stream up to a given
+position.
+
+::
+
+    DistributedLogClient client = ...;
+
+    // Truncate a stream to DLSN
+    String streamName = "test-stream";
+    DLSN truncationDLSN = ...;
+    Future<DLSN> truncateFuture = client.truncate(streamName, truncationDLSN);
+    Await.result(truncateFuture);
+    

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/3469fc87/website/docs/0.4.0-incubating/user_guide/architecture/main.rst
----------------------------------------------------------------------
diff --git a/website/docs/0.4.0-incubating/user_guide/architecture/main.rst b/website/docs/0.4.0-incubating/user_guide/architecture/main.rst
new file mode 100644
index 0000000..9ff455c
--- /dev/null
+++ b/website/docs/0.4.0-incubating/user_guide/architecture/main.rst
@@ -0,0 +1,178 @@
+---
+layout: default
+
+# Top navigation
+top-nav-group: user-guide
+top-nav-pos: 3
+top-nav-title: Architecture
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: user-guide
+sub-nav-id: architecture
+sub-nav-pos: 3
+sub-nav-title: Architecture
+---
+
+.. contents:: Architecture
+
+Architecture
+============
+
+Data Model
+----------
+
+DistributedLog exposes the `log stream` as the unit of operations. A `log stream` is a totally ordered,
+immutable sequence of log records. A `log record` is a sequence of bytes. Log records are batched into `entries`
+and written into `log segments`. Figure 1 illustrates the logical elements of a log stream.
+
+.. figure:: ../../images/datamodel.png
+   :align: center
+
+   Figure 1. Anatomy of a log stream
+
+Log Segments
+~~~~~~~~~~~~
+
+Although the application views the log stream as a continuous sequence of log records, it is physically stored as
+multiple `log segments`, where these segments are the unit of `manageability`. All the records in a log segment have
+the same replication configuration. The log segments are allocated, distributed and stored in a `log segment store`.
+As records are written to the log stream, DistributedLog switches to a new log segment based on a configured `rolling policy`.
+The rolling policy can be `time-based` i.e. based on a configured period of time (e.g. every 2 hours) or `size-based`
+i.e. based on a maximum log segment size (e.g. every 128MB). This allows the log segments to be distributed evenly
+across all the storage nodes. This helps evenly spread read traffic to avoid hot spots in the cluster.
+
+A log segment is also the unit of data retention. Log segments are deleted either by explicitly truncation or expiration.
+Old data is garbage collected by the log segment store once the log segments are deleted.
+
+Log Sequence Numbers
+~~~~~~~~~~~~~~~~~~~~
+
+`Log records` are written sequentially into a log stream, and assigned a unique sequence number called `DLSN`
+(DistributedLog Sequence Number). A DLSN is comprised of 3 components: a `Log Segment Sequence Number` (LSSN),
+the sequence number of the log segment that the record belongs to, an `Entry ID` (EID) - the entry id in this log segment
+that the record is in, and a `Slot ID` (SID) - the slot within the entry. Records can be ordered by DLSN. 
+
+Besides DLSN, an application can assign a `Transaction ID`,  a non-decreasing positive 64-bit integer, to each log record it writes.
+This facilitates application-specific sequencing of records and positioning of the reader. For example, a common use of the transaction ID
+is to store the timestamp of when the log record was added to the log stream. This transaction ID can then be used to rewind to a specific
+time in analytics applications.
+
+Namespace
+~~~~~~~~~
+
+Log streams that belong to same application are usually categorized and managed under a `namespace`. A `namespace` is used by applications
+to locate where the log streams are. Applications can `create` and `delete` streams under a namespace, and `truncate` a stream to given ID.
+
+Software Stack
+--------------
+
+The software stack is shown in Figure 2. The software stack is divided into three layers, with each layer is responsible for
+different features of DistributedLog. These layers are `Persistent Storage`, `DistributedLog Core` and `Stateless Serving`.
+
+.. figure:: ../../images/softwarestack.png
+   :align: center
+
+   Figure 2. DistributedLog Software Stack
+
+Persistent Storage
+~~~~~~~~~~~~~~~~~~
+
+DistributedLog provides the core features - `durability`, `availability` and `consistency` through the storage layer.
+The main components in storage layer are `Log Segment Store`, `Cold Storage` and `Metadata Store`.
+
+Log Segment Store
++++++++++++++++++
+
+The Log segment store is responsible for storing the log segments as they are created and ensure they are durably replicated.
+We use `Apache BookKeeper` as the log segment store. BookKeeper helps us achieve low tail latencies for writes and reads as well as
+low delivery latency which is the end to end latency from when the record is generated until it is read by the reader - because bookkeeper's
+storage layout is optimized for I/O isolation for log workloads.
+
+In addition to storage layout optimization, the log segment store (via BookKeeper) also provides a built-in `fencing` mechanism for
+achieving strong consistency among multiple writers. We will discuss more about consistency in section `Design Details`.
+
+Cold Storage
+++++++++++++
+
+The data in the log segment store is eventually moved to a `cold storage`. Cold storage allows cost efficient storage of large volumes
+of log segments for extended period of time. Applications many want to have access to old data for application error recovery or debugging.
+As log segments are completed, they are proactively copied over to the cold storage, thereby providing a backup for disaster recovery or an
+operation error. We use HDFS as our cold storage.
+
+Metadata Store
+++++++++++++++
+
+The metadata in DistributedLog consists of the mapping from log streams to their constituent log segments as well as each log segment's metadata.
+The log segment metadata includes the `log segment ID`, `start and end transaction IDs`, `completion time`, and its `status`. The metadata store
+is required to provide metadata operations such as consistent read and write ordering to guarantee metadata consistency in the event of failures.
+Also the metadata store should provide a notification mechanism to support streaming reads. We use ZooKeeper as the metadata store, because it is
+a strongly consistent data store which provides versioned updates, strong ordering and notifications using watches.
+
+DistributedLog Core
+~~~~~~~~~~~~~~~~~~~
+
+DistributedLog builds its core functionality on top of the log segment store and the metadata store. It provides the core data model of log streams
+and its naming system, and provides a `single-writer-multiple-reader` access model.
+
+Writers write data into the logs of their choice. Writers sequence log records written to the log streams. Therefore there is only one active log
+segment for a given log stream at a time. Correctness and consistency are guaranteed using a fencing mechanism in the log segment store and
+versioned metadata updates to prevent two writers from writing to the same log stream after a network partition.
+
+Reading from a log stream starts by `positioning` a reader on a log record by specifying either a DLSN or a Transaction Id. Once a reader has been
+positioned, it receives all the log records in increasing order of the sequence numbers and each record is delivered exactly once. It is up to
+individual applications to choose an appropriate mechanism to record readers positions and provide this position when a new reader session begins
+(e.g restart from a failure). Applications can choose the appropriate method for storing positions based on the desired processing semantics.
+
+Stateless Serving
+~~~~~~~~~~~~~~~~~
+
+A stateless serving layer is built on top of the storage layer to support large number of writers and readers. The serving layer includes `Write Proxy`
+and `Read Proxy`. `Write Proxy` manages the ownerships of the log streams, forwards the write requests to storage via the core library and handles
+load balancing and failover. It allows sequencing writes from many clients (aka `Fan-in`). `Read Proxy` caches log records for multiple readers consuming
+the same log stream.
+
+Ownership Tracker
++++++++++++++++++
+
+`Ownership Tracker` tracks the liveness of the owners of the log streams and fails over the ownership to other healthy write proxies when the current
+owner becomes unavailable. Since we already use zookeeper for metadata storage, we also use zookeeper for tracking the liveness of write proxies using
+`ephemeral znodes` and failover the ownership when zookeeper session expires.
+
+Routing Service
++++++++++++++++
+Since readers read committed data and are strict followers, the read proxies do not have to track ownership of log streams. We use consistent hashing
+as a routing mechanism to route the readers to corresponding read proxies.
+
+Applications can either use a thin client that talks to the serving tier to access DistributedLog or embed the core library to talk to the storage directly
+when they require strict write ordering. Applications choose partitioning strategies and track their reader positions based on their specific requirements.
+
+Lifecyle of records
+-------------------
+
+Figure 3 illustrates the lifecycle of a log record in DistributedLog as it flows from writers to readers and we discuss how different layers interact with
+each other.
+
+.. figure:: ../../images/requestflow.png
+   :align: center
+
+   Figure 3. Lifecycle of a record 
+
+The application constructs the log records and initiates write requests (step 1). The write requests will be forwarded to the write proxy that is the master
+of the log stream. The master writer proxy will write the records in the log stream's transmit buffer. Based on the configured transmit policy, records in
+the transmit buffer will be transmitted as a batched entry to log segment store (step 2). Application can trade latency for throughput by transmitting
+`immediately` (lowest latency), `periodically` (grouping records that appear within the transmit period) or when transmit buffer has accumulated more than
+`max-outstanding bytes`.
+
+The batched entry is transmitted to multiple bookies (storage nodes in bookkeeper cluster) in parallel (step 3). The log segment store will respond back to
+writer once the entry is persisted durably on disk. Once the write proxy receives confirmation of durability from a quorum of bookies, it will send an
+acknowledgement to the application (step 4).
+
+Although the writer knows that the record is guaranteed to be persisted and replicated in bookkeeper. Individual bookies do not necessarily know that the
+consensus agreement has been reached for this record. The writer must therefore record a `commit` to make this record visible to all the readers.
+This `commit` can piggyback on the next batch of records from the application. If no new application records are received within the specified SLA for
+persistence, the writer will issue a special `control log record` notifying the log segment store that the record can now be made visible to readers (step 5).
+
+The readers' request that is waiting for new data using `long polling` will now receive the recent committed log records (step 6). Speculative long poll reads will be sent to other replicas to archieve predictable low 99.9% percentile latency (step 7).
+
+The log records will be cached in read proxies (step 8) for fanout readers. The read clients also use similar long poll read mechanism to read data from read proxies (step 9).