You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@distributedlog.apache.org by si...@apache.org on 2016/09/13 07:34:27 UTC

[09/23] incubator-distributedlog git commit: DL-3: Move distributedlog website to apache

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/docs/user_guide/references/features.rst
----------------------------------------------------------------------
diff --git a/docs/user_guide/references/features.rst b/docs/user_guide/references/features.rst
new file mode 100644
index 0000000..c92ae3f
--- /dev/null
+++ b/docs/user_guide/references/features.rst
@@ -0,0 +1,42 @@
+---
+layout: default
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: references
+sub-nav-pos: 3
+sub-nav-title: Available Features
+---
+
+.. contents:: Available Features
+
+Features
+========
+
+BookKeeper Features
+-------------------
+
+*<scope>* is the scope value of the FeatureProvider passed to BookKeeperClient builder. in DistributedLog write proxy, the *<scope>* is 'bkc'.
+
+- *<scope>.repp_disable_durability_enforcement*: Feature to disable durability enforcement on region aware data placement policy. It is a feature that applied for global replicated log only. If the availability value is larger than zero, the region aware data placement policy will *NOT* enfore region-wise durability. That says if a *Log* is writing to region A, B, C with write quorum size *15* and ack quorum size *9*. If the availability value of this feature is zero, it requires *9*
+  acknowledges from bookies from at least two regions. If the availability value of this feature is larger than zero, the enforcement is *disabled* and it could acknowledge after receiving *9* acknowledges from whatever regions. By default the availability is zero. Turning on this value to tolerant multiple region failures.
+
+- *<scope>.disable_ensemble_change*: Feature to disable ensemble change on DistributedLog writers. If the availability value of this feature is larger than zero, it would disable ensemble change on writers. It could be used for toleranting zookeeper outage.
+
+- *<scope>.<region>.disallow_bookie_placement*: Feature to disallow choosing a bookie replacement from a given *region* when ensemble changing. It is a feature that applied for global replicated log. If the availability value is larger than zero, the writer (write proxy) will stop choosing a bookie from *<region>* when ensemble changing. It is useful to blackout a region dynamically.
+
+DistributedLog Features
+-----------------------
+
+*<scope>* is the scope value of the FeatureProvider passed to DistributedLogNamespace builder. in DistributedLog write proxy, the *<scope>* is 'dl'.
+
+- *<scope>.disable_logsegment_rolling*: Feature to disable log segment rolling. If the availability value is larger than zero, the writer (write proxy) will stop rolling to new log segments and keep writing to current log segments. It is a useful feature to tolerant zookeeper outage.
+
+- *<scope>.disable_write_limit*: Feature to disable write limiting. If the availability value is larger than zero, the writer (write proxy) will disable write limiting. It is used to control write limiting dynamically.
+
+Write Proxy Features
+--------------------
+
+- *region_stop_accept_new_stream*: Feature to disable accepting new streams in current region. It is a feature that applied for global replicated log only. If the availability value is larger than zero, the write proxies will stop accepting new streams and throw RegionAvailable exception to client. Hence client will know this region is stopping accepting new streams. Client will be forced to send requests to other regions. It is a feature used for ownership failover between regions.
+- *service_rate_limit_disabled*: Feature to disable service rate limiting. If the availability value is larger than zero, the write proxies will disable rate limiting.
+- *service_checksum_disabled*: Feature to disable service request checksum validation. If the availability value is larger than zero, the write proxies will disable request checksum validation.

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/docs/user_guide/references/main.rst
----------------------------------------------------------------------
diff --git a/docs/user_guide/references/main.rst b/docs/user_guide/references/main.rst
new file mode 100644
index 0000000..b0e62eb
--- /dev/null
+++ b/docs/user_guide/references/main.rst
@@ -0,0 +1,28 @@
+---
+layout: default
+
+# Top navigation
+top-nav-group: user-guide
+top-nav-pos: 9
+top-nav-title: References
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: user-guide
+sub-nav-id: references
+sub-nav-pos: 9
+sub-nav-title: References
+---
+
+References
+===========
+
+This page keeps references on configuration settings, metrics and features that exposed in DistributedLog.
+
+- `Metrics`_
+
+.. _Metrics: ./metrics
+
+- `Available Features`_
+
+.. _Available Features: ./features

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/docs/user_guide/references/metrics.rst
----------------------------------------------------------------------
diff --git a/docs/user_guide/references/metrics.rst b/docs/user_guide/references/metrics.rst
new file mode 100644
index 0000000..9f3c9a3
--- /dev/null
+++ b/docs/user_guide/references/metrics.rst
@@ -0,0 +1,492 @@
+---
+layout: default
+
+# Sub-level navigation
+sub-nav-group: user-guide
+sub-nav-parent: references
+sub-nav-pos: 2
+sub-nav-title: Metrics
+
+---
+
+.. contents:: Metrics
+
+Metrics
+=======
+
+This section lists the metrics exposed by main classes.
+
+({scope} is referencing current scope value of passed in StatsLogger.)
+
+MonitoredFuturePool
+-------------------
+
+**{scope}/tasks_pending**
+
+Gauge. How many tasks are pending in this future pool? If this value becomes high, it means that
+the future pool execution rate couldn't keep up with submission rate. That would be cause high
+*task_pending_time* hence affecting the callers that use this future pool.
+It could also cause heavy jvm gc if this pool keeps building up.
+
+**{scope}/task_pending_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on waiting being executed.
+It becomes high because either *tasks_pending* is building up or *task_execution_time* is high blocking other
+tasks to execute.
+
+**{scope}/task_execution_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on execution. If it becomes high,
+it would block other tasks to execute if there isn't enough threads in this executor, hence cause high
+*task_pending_time* and impact user end latency.
+
+**{scope}/task_enqueue_time**
+
+OpStats. The time that tasks spent on submission. The submission time would also impact user end latency.
+
+MonitoredScheduledThreadPoolExecutor
+------------------------------------
+
+**{scope}/pending_tasks**
+
+Gauge. How many tasks are pending in this thread pool executor? If this value becomes high, it means that
+the thread pool executor execution rate couldn't keep up with submission rate. That would be cause high
+*task_pending_time* hence affecting the callers that use this executor. It could also cause heavy jvm gc if
+queue keeps building up.
+
+**{scope}/completed_tasks**
+
+Gauge. How many tasks are completed in this thread pool executor?
+
+**{scope}/total_tasks**
+
+Gauge. How many tasks are submitted to this thread pool executor?
+
+**{scope}/task_pending_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on waiting being executed.
+It becomes high because either *pending_tasks* is building up or *task_execution_time* is high blocking other
+tasks to execute.
+
+**{scope}/task_execution_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on execution. If it becomes high,
+it would block other tasks to execute if there isn't enough threads in this executor, hence cause high
+*task_pending_time* and impact user end latency.
+
+OrderedScheduler
+----------------
+
+OrderedScheduler is a thread pool based *ScheduledExecutorService*. It is comprised with multiple
+MonitoredScheduledThreadPoolExecutor_. Each MonitoredScheduledThreadPoolExecutor_ is wrapped into a
+MonitoredFuturePool_. So there are aggregated stats and per-executor stats exposed.
+
+Aggregated Stats
+~~~~~~~~~~~~~~~~
+
+**{scope}/task_pending_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on waiting being executed.
+It becomes high because either *pending_tasks* is building up or *task_execution_time* is high blocking other
+tasks to execute.
+
+**{scope}/task_execution_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on execution. If it becomes high,
+it would block other tasks to execute if there isn't enough threads in this executor, hence cause high
+*task_pending_time* and impact user end latency.
+
+**{scope}/futurepool/tasks_pending**
+
+Gauge. How many tasks are pending in this future pool? If this value becomes high, it means that
+the future pool execution rate couldn't keep up with submission rate. That would be cause high
+*task_pending_time* hence affecting the callers that use this future pool.
+It could also cause heavy jvm gc if this pool keeps building up.
+
+**{scope}/futurepool/task_pending_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on waiting being executed.
+It becomes high because either *tasks_pending* is building up or *task_execution_time* is high blocking other
+tasks to execute.
+
+**{scope}/futurepool/task_execution_time**
+
+OpStats. It measures the characteristics about the time that tasks spent on execution. If it becomes high,
+it would block other tasks to execute if there isn't enough threads in this executor, hence cause high
+*task_pending_time* and impact user end latency.
+
+**{scope}/futurepool/task_enqueue_time**
+
+OpStats. The time that tasks spent on submission. The submission time would also impact user end latency.
+
+Per Executor Stats
+~~~~~~~~~~~~~~~~~~
+
+Stats about individual executors are exposed under *{scope}/{name}-executor-{id}-0*. *{name}* is the scheduler
+name and *{id}* is the index of the executor in the pool. The corresponding stats of its futurepool are exposed
+under *{scope}/{name}-executor-{id}-0/futurepool*. See MonitoredScheduledThreadPoolExecutor_ and MonitoredFuturePool_
+for more details.
+
+ZooKeeperClient
+---------------
+
+Operation Stats
+~~~~~~~~~~~~~~~
+
+All operation stats are exposed under {scope}/zk. The stats are **latency** *OpStats*
+on zookeeper operations.
+
+**{scope}/zk/{op}**
+
+latency stats on operations.
+these operations are *create_client*, *get_data*, *set_data*, *delete*, *get_children*, *multi*, *get_acl*, *set_acl* and *sync*.
+
+Watched Event Stats
+~~~~~~~~~~~~~~~~~~~
+
+All stats on zookeeper watched events are exposed under {scope}/watcher. The stats are *Counter* about the watched events that this client received:
+
+**{scope}/watcher/state/{keeper_state}**
+
+the number of `KeeperState` changes that this client received. The states are *Disconnected*, *SyncConnected*,
+*AuthFailed*, *ConnectedReadOnly*, *SaslAuthenticated* and *Expired*. By monitoring metrics like *SyncConnected*
+or *Expired* it would help understanding the healthy of this zookeeper client.
+
+**{scope}/watcher/events/{event}**
+
+the number of `Watcher.Event` received by this client. Those events are *None*, *NodeCreated*, *NodeDeleted*,
+*NodeDataChanged*, *NodeChildrenChanged*.
+
+Watcher Manager Stats
+~~~~~~~~~~~~~~~~~~~~~
+
+This ZooKeeperClient provides a watcher manager to manage watchers for applications. It tracks the mapping between
+paths and watcher. It is the way to provide the ability on removing watchers. The stats are *Gauge* about the number
+of watchers managed by this zookeeper client.
+
+**{scope}/watcher_manager/total_watches**
+
+total number of watches that are managed by this watcher manager. If it keeps growing, it usually means that
+watchers are leaking (resources aren't closed properly). It will cause OOM.
+
+**{scope}/watcher_manager/num_child_watches**
+
+total number of paths that are watched by this watcher manager.
+
+BookKeeperClient
+----------------
+
+TODO: add bookkeeper stats there
+
+DistributedReentrantLock
+------------------------
+
+All stats related to locks are exposed under {scope}/lock.
+
+**{scope}/acquire**
+
+OpStats. It measures the characteristics about the time that spent on acquiring locks.
+
+**{scope}/release**
+
+OpStats. It measures the characteristics about the time that spent on releasing locks.
+
+**{scope}/reacquire**
+
+OpStats. The lock will be expired when the underneath zookeeper session expired. The
+reentrant lock will attempt to re-acquire the lock automatically when session expired.
+This metric measures the characteristics about the time that spent on re-acquiring locks.
+
+**{scope}/internalTryRetries**
+
+Counter. The number of retries that locks spend on re-creating internal locks. Typically,
+a new internal lock will be created when session expired.
+
+**{scope}/acquireTimeouts**
+
+Counter. The number of timeouts that caller experienced when acquiring locks.
+
+**{scope}/tryAcquire**
+
+OpStats. It measures the characteristics about the time that each internal lock spent on
+acquiring.
+
+**{scope}/tryTimeouts**
+
+Counter. The number of timeouts that internal locks try acquiring.
+
+**{scope}/unlock**
+
+OpStats. It measures the characteristics about the time that the caller spent on unlocking
+internal locks.
+
+BKLogHandler
+------------
+
+The log handler is a base class on managing log segments. so all the metrics in this class are
+related log segments retrieval and exposed under {scope}/logsegments. They are all `OpStats` in
+the format of `{scope}/logsegments/{op}`. Those operations are:
+
+* force_get_list: force to get the list of log segments.
+* get_list: get the list of the log segments. it might just retrieve from local log segment cache.
+* get_filtered_list: get the filtered list of log segments.
+* get_full_list: get the full list of log segments.
+* get_inprogress_segment: time between the inprogress log segment created and the handler read it.
+* get_completed_segment: time between a log segment is turned to completed and the handler read it.
+* negative_get_inprogress_segment: record the negative values for `get_inprogress_segment`.
+* negative_get_completed_segment: record the negative values for `get_completed_segment`.
+* recover_last_entry: recovering last entry from a log segment.
+* recover_scanned_entries: the number of entries that are scanned during recovering.
+
+See BKLogWriteHandler_ for write handlers.
+
+See BKLogReadHandler_ for read handlers.
+
+BKLogReadHandler
+----------------
+
+The core logic in log reader handle is readahead worker. Most of readahead stats are exposed under
+{scope}/readahead_worker.
+
+**{scope}/readahead_worker/wait**
+
+Counter. Number of waits that readahead worker is waiting. If this keeps increasing, it usually means
+readahead keep getting full because of reader slows down reading.
+
+**{scope}/readahead_worker/repositions**
+
+Counter. Number of repositions that readhead worker encounters. Reposition means that a readahead worker
+finds that it isn't advancing to a new log segment and force re-positioning.
+
+**{scope}/readahead_worker/entry_piggy_back_hits**
+
+Counter. It increases when the last add confirmed being advanced because of the piggy-back lac.
+
+**{scope}/readahead_worker/entry_piggy_back_misses**
+
+Counter. It increases when the last add confirmed isn't advanced by a read entry because it doesn't
+iggy back a newer lac.
+
+**{scope}/readahead_worker/read_entries**
+
+OpStats. Stats on number of entries read per readahead read batch.
+
+**{scope}/readahead_worker/read_lac_counter**
+
+Counter. Stats on the number of readLastConfirmed operations
+
+**{scope}/readahead_worker/read_lac_and_entry_counter**
+
+Counter. Stats on the number of readLastConfirmedAndEntry operations.
+
+**{scope}/readahead_worker/cache_full**
+
+Counter. It increases each time readahead worker finds cache become full. If it keeps increasing,
+that means reader slows down reading.
+
+**{scope}/readahead_worker/resume**
+
+OpStats. Stats on readahead worker resuming reading from wait state.
+
+**{scope}/readahead_worker/long_poll_interruption**
+
+OpStats. Stats on the number of interruptions happened to long poll. the interruptions are usually
+because of receiving zookeeper notifications.
+
+**{scope}/readahead_worker/notification_execution**
+
+OpStats. Stats on executions over the notifications received from zookeeper.
+
+**{scope}/readahead_worker/metadata_reinitialization**
+
+OpStats. Stats on metadata reinitialization after receiving notifcation from log segments updates.
+
+**{scope}/readahead_worker/idle_reader_warn**
+
+Counter. It increases each time the readahead worker detects itself becoming idle.
+
+BKLogWriteHandler
+-----------------
+
+Log write handlers are responsible for log segment creation/deletions. All the metrics are exposed under
+{scope}/segments.
+
+**{scope}/segments/open**
+
+OpStats. Latency characteristics on starting a new log segment.
+
+**{scope}/segments/close**
+
+OpStats. Latency characteristics on completing an inprogress log segment.
+
+**{scope}/segments/recover**
+
+OpStats. Latency characteristics on recovering a log segment.
+
+**{scope}/segments/delete**
+
+OpStats. Latency characteristics on deleting a log segment.
+
+BKAsyncLogWriter
+----------------
+
+**{scope}/log_writer/write**
+
+OpStats. latency characteristics about the time that write operations spent.
+
+**{scope}/log_writer/write/queued**
+
+OpStats. latency characteristics about the time that write operations spent in the queue.
+`{scope}/log_writer/write` latency is high might because the write operations are pending
+in the queue for long time due to log segment rolling.
+
+**{scope}/log_writer/bulk_write**
+
+OpStats. latency characteristics about the time that bulk_write operations spent.
+
+**{scope}/log_writer/bulk_write/queued**
+
+OpStats. latency characteristics about the time that bulk_write operations spent in the queue.
+`{scope}/log_writer/bulk_write` latency is high might because the write operations are pending
+in the queue for long time due to log segment rolling.
+
+**{scope}/log_writer/get_writer**
+
+OpStats. the time spent on getting the writer. it could spike when there is log segment rolling
+happened during getting the writer. it is a good stat to look into when the latency is caused by
+queuing time.
+
+**{scope}/log_writer/pending_request_dispatch**
+
+Counter. the number of queued operations that are dispatched after log segment is rolled. it is
+an metric on measuring how many operations has been queued because of log segment rolling.
+
+BKAsyncLogReader
+----------------
+
+**{scope}/async_reader/future_set**
+
+OpStats. Time spent on satisfying futures of read requests. if it is high, it means that the caller
+takes time on processing the result of read requests. The side effect is blocking consequent reads.
+
+**{scope}/async_reader/schedule**
+
+OpStats. Time spent on scheduling next reads.
+
+**{scope}/async_reader/background_read**
+
+OpStats. Time spent on background reads.
+
+**{scope}/async_reader/read_next_exec**
+
+OpStats. Time spent on executing `reader#readNext()`
+
+**{scope}/async_reader/time_between_read_next**
+
+OpStats. Time spent on between two consequent `reader#readNext()`. if it is high, it means that
+the caller is slowing down on calling `reader#readNext()`.
+
+**{scope}/async_reader/delay_until_promise_satisfied**
+
+OpStats. Total latency for the read requests.
+
+**{scope}/async_reader/idle_reader_error**
+
+Counter. The number idle reader errors.
+
+BKDistributedLogManager
+-----------------------
+
+Future Pools
+~~~~~~~~~~~~
+
+The stats about future pools that used by writers are exposed under {scope}/writer_future_pool,
+while the stats about future pools that used by readers are exposed under {scope}/reader_future_pool.
+See MonitoredFuturePool_ for detail stats.
+
+Distributed Locks
+~~~~~~~~~~~~~~~~~
+
+The stats about the locks used by writers are exposed under {scope}/lock while those used by readers
+are exposed under {scope}/read_lock/lock. See DistributedReentrantLock_ for detail stats.
+
+Log Handlers
+~~~~~~~~~~~~
+
+**{scope}/logsegments**
+
+All basic stats of log handlers are exposed under {scope}/logsegments. See BKLogHandler_ for detail stats.
+
+**{scope}/segments**
+
+The stats about write log handlers are exposed under {scope}/segments. See BKLogWriteHandler_ for detail stats.
+
+**{scope}/readhead_worker**
+
+The stats about read log handlers are exposed under {scope}/readahead_worker.
+See BKLogReadHandler_ for detail stats.
+
+Writers
+~~~~~~~
+
+All writer related metrics are exposed under {scope}/log_writer. See BKAsyncLogWriter_ for detail stats.
+
+Readers
+~~~~~~~
+
+All reader related metrics are exposed under {scope}/async_reader. See BKAsyncLogReader_ for detail stats.
+
+BKDistributedLogNamespace
+-------------------------
+
+ZooKeeper Clients
+~~~~~~~~~~~~~~~~~
+
+There are various of zookeeper clients created per namespace for different purposes. They are:
+
+**{scope}/dlzk_factory_writer_shared**
+
+Stats about the zookeeper client shared by all DL writers.
+
+**{scope}/dlzk_factory_reader_shared**
+
+Stats about the zookeeper client shared by all DL readers.
+
+**{scope}/bkzk_factory_writer_shared**
+
+Stats about the zookeeper client used by bookkeeper client that shared by all DL writers.
+
+**{scope}/bkzk_factory_reader_shared**
+
+Stats about the zookeeper client used by bookkeeper client that shared by all DL readers.
+
+See ZooKeeperClient_ for zookeeper detail stats.
+
+BookKeeper Clients
+~~~~~~~~~~~~~~~~~~
+
+All the bookkeeper client related stats are exposed directly to current {scope}. See BookKeeperClient_
+for detail stats.
+
+Utils
+~~~~~
+
+**{scope}/factory/thread_pool**
+
+Stats about the ordered scheduler used by this namespace. See OrderedScheduler_ for detail stats.
+
+**{scope}/factory/readahead_thread_pool**
+
+Stats about the readahead thread pool executor used by this namespace. See MonitoredScheduledThreadPoolExecutor_
+for detail stats.
+
+**{scope}/writeLimiter**
+
+Stats about the global write limiter used by list namespace.
+
+DistributedLogManager
+~~~~~~~~~~~~~~~~~~~~~
+
+All the core stats about reader and writer are exposed under current {scope} via BKDistributedLogManager_.
+
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/.gitignore
----------------------------------------------------------------------
diff --git a/website/.gitignore b/website/.gitignore
new file mode 100644
index 0000000..c28a8b0
--- /dev/null
+++ b/website/.gitignore
@@ -0,0 +1,5 @@
+_site/
+.sass-cache/
+.jekyll-metadata
+_pdf
+.idea/

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/Gemfile
----------------------------------------------------------------------
diff --git a/website/Gemfile b/website/Gemfile
new file mode 100644
index 0000000..8bc5038
--- /dev/null
+++ b/website/Gemfile
@@ -0,0 +1,5 @@
+source 'https://rubygems.org'
+gem 'jekyll'
+gem 'jekyll-redirect-from'
+gem 'html-proofer'
+gem 'RbST'

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/Gemfile.lock
----------------------------------------------------------------------
diff --git a/website/Gemfile.lock b/website/Gemfile.lock
new file mode 100644
index 0000000..136ae6f
--- /dev/null
+++ b/website/Gemfile.lock
@@ -0,0 +1,83 @@
+GEM
+  remote: https://rubygems.org/
+  specs:
+    RbST (0.5.1)
+    activesupport (4.2.7)
+      i18n (~> 0.7)
+      json (~> 1.7, >= 1.7.7)
+      minitest (~> 5.1)
+      thread_safe (~> 0.3, >= 0.3.4)
+      tzinfo (~> 1.1)
+    addressable (2.4.0)
+    colorator (1.1.0)
+    colored (1.2)
+    ethon (0.9.0)
+      ffi (>= 1.3.0)
+    ffi (1.9.14)
+    forwardable-extended (2.6.0)
+    html-proofer (3.0.6)
+      activesupport (~> 4.2)
+      addressable (~> 2.3)
+      colored (~> 1.2)
+      mercenary (~> 0.3.2)
+      nokogiri (~> 1.5)
+      parallel (~> 1.3)
+      typhoeus (~> 0.7)
+      yell (~> 2.0)
+    i18n (0.7.0)
+    jekyll (3.2.1)
+      colorator (~> 1.0)
+      jekyll-sass-converter (~> 1.0)
+      jekyll-watch (~> 1.1)
+      kramdown (~> 1.3)
+      liquid (~> 3.0)
+      mercenary (~> 0.3.3)
+      pathutil (~> 0.9)
+      rouge (~> 1.7)
+      safe_yaml (~> 1.0)
+    jekyll-redirect-from (0.11.0)
+      jekyll (>= 2.0)
+    jekyll-sass-converter (1.4.0)
+      sass (~> 3.4)
+    jekyll-watch (1.5.0)
+      listen (~> 3.0, < 3.1)
+    json (1.8.3)
+    kramdown (1.11.1)
+    liquid (3.0.6)
+    listen (3.0.8)
+      rb-fsevent (~> 0.9, >= 0.9.4)
+      rb-inotify (~> 0.9, >= 0.9.7)
+    mercenary (0.3.6)
+    mini_portile2 (2.1.0)
+    minitest (5.9.0)
+    nokogiri (1.6.8)
+      mini_portile2 (~> 2.1.0)
+      pkg-config (~> 1.1.7)
+    parallel (1.9.0)
+    pathutil (0.14.0)
+      forwardable-extended (~> 2.6)
+    pkg-config (1.1.7)
+    rb-fsevent (0.9.7)
+    rb-inotify (0.9.7)
+      ffi (>= 0.5.0)
+    rouge (1.11.1)
+    safe_yaml (1.0.4)
+    sass (3.4.22)
+    thread_safe (0.3.5)
+    typhoeus (0.8.0)
+      ethon (>= 0.8.0)
+    tzinfo (1.2.2)
+      thread_safe (~> 0.1)
+    yell (2.0.6)
+
+PLATFORMS
+  ruby
+
+DEPENDENCIES
+  RbST
+  html-proofer
+  jekyll
+  jekyll-redirect-from
+
+BUNDLED WITH
+   1.12.5

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/README.md
----------------------------------------------------------------------
diff --git a/website/README.md b/website/README.md
new file mode 100644
index 0000000..f07ff10
--- /dev/null
+++ b/website/README.md
@@ -0,0 +1,79 @@
+# Apache DistributedLog (incubating) website
+
+This is the website for [Apache DistributedLog](http://distributedlog.incubator.apache.org)
+(incubating).
+
+### About this site
+The DistributedLog website is built using [Jekyll](http://jekyllrb.com/). Additionally,
+for additional formatting capabilities, this website uses
+[Twitter Bootstrap](http://getbootstrap.com/).
+
+This website is hosted at:
+
+    http://distributedlog.incubator.apache.org
+
+It is important to note there are two sets of "website code"  with respect to
+the Apache DistributedLog website.
+
+1. The **Jekyll code** which contains all of the resources for building,
+testing, styling, and tinkering with the website. Think of it as a website SDK.
+1. The **static website** content which contains the content for the
+website. This is the static content is what is actually hosted on the Apache 
+DistributedLog website.
+
+### Development setup
+Before working with the Jekyll code, you will need to install Jekyll:
+
+    $ gem install jekyll
+    $ gem install jekyll-redirect-from
+    $ gem install html-proofer
+
+*If you are on a Mac, you may need to install
+[Ruby Gems](https://rubygems.org/pages/download).*
+
+### Live development
+While you are working with the website, you can test and develop live. Run the
+following command in the root folder of the website:
+
+    $ jekyll serve
+
+Jekyll will start a webserver on port `4000`. As you make changes to the
+content, Jekyll will rebuild it automatically. This is helpful if you want to see
+how your changes will render in realtime.
+
+In addition, you can run the tests via:
+
+    $ rake test
+
+### Generating the static website
+Once you are done with your changes, you need to compile the static
+content for the website. This is what is actually hosted 
+on the Apache DistributedLog website.
+
+You can build the static content by running the following command in the root
+website directory:
+
+    $ jekyll build
+
+Once built, it will be placed in the folder `content` inside of the root directory. 
+This directory will include images, HTML, CSS, and so on. In a typical Jekyll install
+this content would live in `_site` - it has been changed for the Apache DistributedLog website
+to work with the ASF Incubator publishing system.
+
+### Apache License
+---
+Except as otherwise noted this software is licensed under the
+[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html)
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/Rakefile
----------------------------------------------------------------------
diff --git a/website/Rakefile b/website/Rakefile
new file mode 100644
index 0000000..9778c98
--- /dev/null
+++ b/website/Rakefile
@@ -0,0 +1,10 @@
+require 'html-proofer'
+
+task :test do
+  sh "bundle exec jekyll build"
+  HTMLProofer.check_directory("./content", {
+    :allow_hash_href => true,
+    :check_html => true,
+    :file_ignore => [/javadoc/]
+    }).run
+end

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_config-local.yml
----------------------------------------------------------------------
diff --git a/website/_config-local.yml b/website/_config-local.yml
new file mode 100644
index 0000000..ca29cd2
--- /dev/null
+++ b/website/_config-local.yml
@@ -0,0 +1 @@
+baseurl: "https://sijie.github.io/incubator-distributedlog/"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_config-production.yml
----------------------------------------------------------------------
diff --git a/website/_config-production.yml b/website/_config-production.yml
new file mode 100644
index 0000000..c63e3ec
--- /dev/null
+++ b/website/_config-production.yml
@@ -0,0 +1 @@
+baseurl: "http://distributedlog.incubator.apache.org"

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_config.yml
----------------------------------------------------------------------
diff --git a/website/_config.yml b/website/_config.yml
new file mode 100644
index 0000000..d3e460c
--- /dev/null
+++ b/website/_config.yml
@@ -0,0 +1,50 @@
+# Welcome to Jekyll!
+#
+# This config file is meant for settings that affect your whole blog, values
+# which you are expected to set up once and rarely need to edit after that.
+# For technical reasons, this file is *NOT* reloaded automatically when you use
+# 'jekyll serve'. If you change this file, please restart the server process.
+
+# Site settings
+title: Apache DistributedLog (incubating)
+description: > # this means to ignore newlines until "baseurl:"
+  Apache DistributedLog is an high performance replicated log.
+
+# versions
+DL_VERSION_STABLE: 0.3.51-RC1
+
+# the subpath of your site, e.g. /blog
+baseurl: ""
+
+# the base hostname & protocol for your site
+url: "http://distributedlog.incubator.apache.org"
+
+twitter_username: distributedlog
+twitter_url: https://twitter.com/distributedlog
+github_url: https://github.com/apache/incubator-distributedlog
+wiki_url: https://cwiki.apache.org/confluence/display/DL/Apache+DistributedLog+Home
+jira_url: https://issues.apache.org/jira/browse/DL
+developer_setup_url: https://cwiki.apache.org/confluence/display/DL/Developer+Setup
+contribute_how_to_url: https://cwiki.apache.org/confluence/display/DL/Contributing+to+DistributedLog
+coding_guide_url: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65867477
+
+# Build settings
+markdown: kramdown
+
+collections:
+- distributedlog_team
+- mentors
+
+# Things to ignore in the build
+exclude: ['README.md', 'Gemfile.lock', 'Gemfile', 'Rakefile', 'docs', 'build.sh']
+
+# Things to keep in the build
+keep_files: ['docs']
+
+# Downloads directory
+downloads: downloads
+
+tracking_id: UA-83870961-1
+
+gems:
+  - jekyll-redirect-from

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_data/authors.yml
----------------------------------------------------------------------
diff --git a/website/_data/authors.yml b/website/_data/authors.yml
new file mode 100644
index 0000000..aaa83ae
--- /dev/null
+++ b/website/_data/authors.yml
@@ -0,0 +1,4 @@
+sijie:
+    name: Sijie Guo
+    email: sijie@apache.org
+    twitter: sijieg

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_data/logos.yml
----------------------------------------------------------------------
diff --git a/website/_data/logos.yml b/website/_data/logos.yml
new file mode 100644
index 0000000..123ac03
--- /dev/null
+++ b/website/_data/logos.yml
@@ -0,0 +1,8 @@
+sizes: [100, 200, 500, 1000]
+colors:
+  full-color: "Full color"
+  3-color: "Three color"
+  bw: "Black and white"
+types: ['nameless', 'name-bottom', 'name-right']
+logo-location: '/images/logos'
+archive-file: logos.zip

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_distributedlog_team/team.md
----------------------------------------------------------------------
diff --git a/website/_distributedlog_team/team.md b/website/_distributedlog_team/team.md
new file mode 100644
index 0000000..6f47569
--- /dev/null
+++ b/website/_distributedlog_team/team.md
@@ -0,0 +1,59 @@
+---
+group: "Committers"
+description: "The following is a list of developers with commit privileges that have directly contributed to the project in one way or another."
+members:
+  - name: Aniruddha Laud
+    apache_id: 
+    email:
+    organization: MixPanel
+    roles: committer
+    time_zone: "-8"
+  - name: Dave Rusek
+    apache_id: drusek
+    email: drusek [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Eitan Adler
+    apache_id: eax
+    email: eax [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Franck Cuny
+    apache_id: fcuny
+    email: fcuny [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Jordan Bull
+    apache_id: 
+    email: 
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Leigh Stewart
+    apache_id: lstewart
+    email: lstewart [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Robin Dhamankar
+    apache_id: robindh
+    email: robindh [at] apache [dot] org
+    organization: Facebook
+    roles: committer
+    time_zone: "-8"
+  - name: Satish Kotha
+    apache_id: satish
+    email: satish [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+  - name: Sijie Guo
+    apache_id: sijie
+    email: sijie [at] apache [dot] org
+    organization: Twitter
+    roles: committer
+    time_zone: "-8"
+---

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/authors-list.md
----------------------------------------------------------------------
diff --git a/website/_includes/authors-list.md b/website/_includes/authors-list.md
new file mode 100644
index 0000000..fb2808e
--- /dev/null
+++ b/website/_includes/authors-list.md
@@ -0,0 +1 @@
+{% assign count = authors | size %}{% for name in authors %}{% if forloop.first == false and count > 2 %},{% endif %}{% if forloop.last and count > 1 %} &amp;{% endif %}{% assign author = site.data.authors[name] %} {{ author.name }} {% if author.twitter %}[<a href="https://twitter.com/{{ author.twitter }}">@{{ author.twitter }}</a>]{% endif %}{% endfor %}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/footer.html
----------------------------------------------------------------------
diff --git a/website/_includes/footer.html b/website/_includes/footer.html
new file mode 100644
index 0000000..9f5f894
--- /dev/null
+++ b/website/_includes/footer.html
@@ -0,0 +1,15 @@
+<hr>
+  <div class="row">
+      <div class="col-xs-12">
+          <footer>
+              <p class="text-center">&copy; Copyright 2016
+                  <a href="http://www.apache.org">The Apache Software Foundation.</a> All Rights Reserved.
+              </p>
+              <p class="text-center">
+                  <a href="{{ "/feed.xml" | prepend: site.baseurl }}">RSS Feed</a>
+              </p>
+          </footer>
+      </div>
+  </div>
+  <!-- container div end -->
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/head.html
----------------------------------------------------------------------
diff --git a/website/_includes/head.html b/website/_includes/head.html
new file mode 100644
index 0000000..13dcd36
--- /dev/null
+++ b/website/_includes/head.html
@@ -0,0 +1,26 @@
+<head>
+  <meta charset="utf-8">
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1">
+
+  <title>{% if page.title %}{{ page.title | escape }}{% else %}{{ site.title | escape }}{% endif %}</title>
+  <meta name="description" content="{% if page.excerpt %}{{ page.excerpt | strip_html | strip_newlines | truncate: 160 }}{% else %}{{ site.description }}{% endif %}">
+
+  <link rel="stylesheet" href="{{ "/styles/site.css" | prepend: site.baseurl }}">
+  <link rel="stylesheet" href="{{ "/css/theme.css" | prepend: site.baseurl }}">
+  <script src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.0/jquery.min.js"></script>
+  <script src="{{ "/js/bootstrap.min.js" | prepend: site.baseurl }}"></script>
+  <link rel="canonical" href="{{ page.url | replace:'index.html','' | prepend: site.baseurl | prepend: site.url }}" data-proofer-ignore>
+  <link rel="alternate" type="application/rss+xml" title="{{ site.title }}" href="{{ "/feed.xml" | prepend: site.baseurl | prepend: site.url }}">
+  <script>
+  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
+  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+  })(window,document,'script','https://www.google-analytics.com/analytics.js','ga');
+
+  ga('create', '{{ site.tracking_id }}', 'auto');
+  ga('send', 'pageview');
+
+  </script> 
+  <link rel="shortcut icon" type="image/x-icon" href="/images/favicon.ico">
+</head>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/header.html
----------------------------------------------------------------------
diff --git a/website/_includes/header.html b/website/_includes/header.html
new file mode 100644
index 0000000..05d1922
--- /dev/null
+++ b/website/_includes/header.html
@@ -0,0 +1,117 @@
+<nav class="navbar navbar-default navbar-fixed-top">
+  <div class="container">
+    <div class="navbar-header">
+      <a href="/" class="navbar-brand" >
+        <img alt="Brand" style="height: 28px" src="{{ "/images/distributedlog_logo_navbar.png" | prepend: site.baseurl }}">
+      </a>
+      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar" aria-expanded="false" aria-controls="navbar">
+        <span class="sr-only">Toggle navigation</span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+        <span class="icon-bar"></span>
+      </button>
+    </div>
+    <div id="navbar" class="navbar-collapse collapse">
+      <ul class="nav navbar-nav">
+        <!-- Overview -->
+        <li><a href="{{ site.baseurl }}/docs/latest/basics/introduction">Overview</a></li>
+        <!-- Downloads -->
+        <li><a href="{{ site.baseurl }}/docs/latest/start/download">Downloads</a></li>
+        <!-- Quick Start -->
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Quick Start<span class="caret"></span></a>
+          <ul class="dropdown-menu">
+            <li><a href="{{ site.baseurl }}/docs/latest/start/quickstart">Setup & Run Example</a></li>
+            <li role="separator" class="divider"></li>
+            <li class="dropdown-header">Tutorials</li>
+            <li>
+              <a href="{{ site.baseurl }}/docs/latest/tutorials/main#id3">
+              <small><span class="glyphicon glyphicon-pencil"></span></small>
+              Basic
+              </a>
+            </li>
+            <li>
+              <a href="{{ site.baseurl }}/docs/latest/tutorials/main#id4">
+              <small><span class="glyphicon glyphicon-envelope"></span></small>
+              Messaging
+              </a>
+            </li>
+            <li>
+              <a href="{{ site.baseurl }}/docs/latest/tutorials/main#id6">
+              <small><span class="glyphicon glyphicon-stats"></span></small>
+              Analytics
+              </a>
+            </li>
+          </ul>
+        </li>
+        <!-- Documentation -->
+        <li class="dropdown">
+		      <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Documentation<span class="caret"></span></a>
+          <ul class="dropdown-menu">
+            <li class="dropdown-header">Snapshot (Developement)</li>
+            <li><a href="{{ site.baseurl }}/docs/latest">Latest</a></li>
+            <li role="separator" class="divider"></li>
+            <li>
+              <a href="https://cwiki.apache.org/confluence/display/DL/Project+Ideas">
+                <small><span class="glyphicon glyphicon-new-window"></span></small>
+                Project Ideas
+              </a>
+            </li>
+          </ul>
+        </li>
+        <!-- FAQ -->
+        <li><a href="{{ site.baseurl }}/faq">FAQ</a></li>
+      </ul>
+      <!-- Right Side -->
+      <ul class="nav navbar-nav navbar-right">
+        <!-- Blog -->
+        <li><a href="{{ site.baseurl }}/blog">Blog</a></li>
+        <!-- Community -->
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Community<span class="caret"></span></a>
+          <ul class="dropdown-menu">
+            <li class="dropdown-header">Community</li>
+            <li><a href="{{ site.baseurl }}/community/#mailing-lists">Mailing Lists</a></li>
+            <li><a href="{{ site.baseurl }}/community/#source-code">Source Code</a></li>
+            <li><a href="{{ site.baseurl }}/community/#issue-tracker">Issue Tracking</a></li>
+            <li><a href="{{ site.baseurl }}/community/team/">Team</a></li>
+            <li role="separator" class="divider"></li>
+            <li class="dropdown-header">Contribute</li>
+            <li><a href="{{ site.developer_setup_url }}">Developer Setup</a></li>
+            <li><a href="{{ site.contribute_how_to_url }}">Contributing to DistributedLog</a></li>
+            <li><a href="{{ site.coding_guide_url }}">Coding Guide</a></li>
+          </ul>
+        </li>
+        <!-- Project -->
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Project<span class="caret"></span></a>
+          <ul class="dropdown-menu">
+			      <li class="dropdown-header">Project</li>
+            <li><a href="{{ site.baseurl }}/project/presentations/">Presentations</a></li>
+            <li>
+              <a href="{{ site.twitter_url }}">
+                <small><span class="glyphicon glyphicon-new-window"></span></small>
+                Twitter
+              </a>
+            </li>
+            <li>
+              <a href="{{ site.github_url }}">
+                <small><span class="glyphicon glyphicon-new-window"></span></small>
+                Github
+              </a>
+            </li>
+            <li>
+              <a href="{{ site.wiki_url }}">
+                <small><span class="glyphicon glyphicon-new-window"></span></small>
+                Wiki
+              </a>
+            </li>
+          </ul>
+        </li>
+      </ul>
+    </div><!--/.nav-collapse -->
+  </div>
+</nav>
+
+
+<link rel="stylesheet" href="">

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/icon-github.html
----------------------------------------------------------------------
diff --git a/website/_includes/icon-github.html b/website/_includes/icon-github.html
new file mode 100644
index 0000000..e501a16
--- /dev/null
+++ b/website/_includes/icon-github.html
@@ -0,0 +1 @@
+<a href="https://github.com/{{ include.username }}"><span class="icon icon--github">{% include icon-github.svg %}</span><span class="username">{{ include.username }}</span></a>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/icon-github.svg
----------------------------------------------------------------------
diff --git a/website/_includes/icon-github.svg b/website/_includes/icon-github.svg
new file mode 100644
index 0000000..4422c4f
--- /dev/null
+++ b/website/_includes/icon-github.svg
@@ -0,0 +1 @@
+<svg viewBox="0 0 16 16"><path fill="#828282" d="M7.999,0.431c-4.285,0-7.76,3.474-7.76,7.761 c0,3.428,2.223,6.337,5.307,7.363c0.388,0.071,0.53-0.168,0.53-0.374c0-0.184-0.007-0.672-0.01-1.32 c-2.159,0.469-2.614-1.04-2.614-1.04c-0.353-0.896-0.862-1.135-0.862-1.135c-0.705-0.481,0.053-0.472,0.053-0.472 c0.779,0.055,1.189,0.8,1.189,0.8c0.692,1.186,1.816,0.843,2.258,0.645c0.071-0.502,0.271-0.843,0.493-1.037 C4.86,11.425,3.049,10.76,3.049,7.786c0-0.847,0.302-1.54,0.799-2.082C3.768,5.507,3.501,4.718,3.924,3.65 c0,0,0.652-0.209,2.134,0.796C6.677,4.273,7.34,4.187,8,4.184c0.659,0.003,1.323,0.089,1.943,0.261 c1.482-1.004,2.132-0.796,2.132-0.796c0.423,1.068,0.157,1.857,0.077,2.054c0.497,0.542,0.798,1.235,0.798,2.082 c0,2.981-1.814,3.637-3.543,3.829c0.279,0.24,0.527,0.713,0.527,1.437c0,1.037-0.01,1.874-0.01,2.129 c0,0.208,0.14,0.449,0.534,0.373c3.081-1.028,5.302-3.935,5.302-7.362C15.76,3.906,12.285,0.431,7.999,0.431z"/></svg>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/icon-twitter.html
----------------------------------------------------------------------
diff --git a/website/_includes/icon-twitter.html b/website/_includes/icon-twitter.html
new file mode 100644
index 0000000..e623dbd
--- /dev/null
+++ b/website/_includes/icon-twitter.html
@@ -0,0 +1 @@
+<a href="https://twitter.com/{{ include.username }}"><span class="icon icon--twitter">{% include icon-twitter.svg %}</span><span class="username">{{ include.username }}</span></a>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_includes/icon-twitter.svg
----------------------------------------------------------------------
diff --git a/website/_includes/icon-twitter.svg b/website/_includes/icon-twitter.svg
new file mode 100644
index 0000000..dcf660e
--- /dev/null
+++ b/website/_includes/icon-twitter.svg
@@ -0,0 +1 @@
+<svg viewBox="0 0 16 16"><path fill="#828282" d="M15.969,3.058c-0.586,0.26-1.217,0.436-1.878,0.515c0.675-0.405,1.194-1.045,1.438-1.809c-0.632,0.375-1.332,0.647-2.076,0.793c-0.596-0.636-1.446-1.033-2.387-1.033c-1.806,0-3.27,1.464-3.27,3.27 c0,0.256,0.029,0.506,0.085,0.745C5.163,5.404,2.753,4.102,1.14,2.124C0.859,2.607,0.698,3.168,0.698,3.767 c0,1.134,0.577,2.135,1.455,2.722C1.616,6.472,1.112,6.325,0.671,6.08c0,0.014,0,0.027,0,0.041c0,1.584,1.127,2.906,2.623,3.206 C3.02,9.402,2.731,9.442,2.433,9.442c-0.211,0-0.416-0.021-0.615-0.059c0.416,1.299,1.624,2.245,3.055,2.271 c-1.119,0.877-2.529,1.4-4.061,1.4c-0.264,0-0.524-0.015-0.78-0.046c1.447,0.928,3.166,1.469,5.013,1.469 c6.015,0,9.304-4.983,9.304-9.304c0-0.142-0.003-0.283-0.009-0.423C14.976,4.29,15.531,3.714,15.969,3.058z"/></svg>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_layouts/default.html
----------------------------------------------------------------------
diff --git a/website/_layouts/default.html b/website/_layouts/default.html
new file mode 100644
index 0000000..b7b9c90
--- /dev/null
+++ b/website/_layouts/default.html
@@ -0,0 +1,21 @@
+<!DOCTYPE html>
+<html lang="en">
+
+  {% include head.html %}
+
+  <body role="document">
+
+    {% include header.html %}
+
+    <div class="container" role="main">
+
+      <div class="row">
+        {{ content }}
+      </div>
+
+
+    {% include footer.html %}
+
+  </body>
+
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_layouts/page.html
----------------------------------------------------------------------
diff --git a/website/_layouts/page.html b/website/_layouts/page.html
new file mode 100644
index 0000000..ce233ad
--- /dev/null
+++ b/website/_layouts/page.html
@@ -0,0 +1,14 @@
+---
+layout: default
+---
+<article class="post">
+
+  <header class="post-header">
+    <h1 class="post-title">{{ page.title }}</h1>
+  </header>
+
+  <div class="post-content">
+    {{ content }}
+  </div>
+
+</article>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_layouts/post.html
----------------------------------------------------------------------
diff --git a/website/_layouts/post.html b/website/_layouts/post.html
new file mode 100644
index 0000000..696bad2
--- /dev/null
+++ b/website/_layouts/post.html
@@ -0,0 +1,17 @@
+---
+layout: default
+---
+{% assign authors = page.authors %}
+
+<article class="post" itemscope itemtype="http://schema.org/BlogPosting">
+
+  <header class="post-header">
+    <h1 class="post-title" itemprop="name headline">{{ page.title }}</h1>
+    <p class="post-meta"><time datetime="{{ page.date | date_to_xmlschema }}" itemprop="datePublished">{{ page.date | date: "%b %-d, %Y" }}</time>{% if authors %} \u2022 {% include authors-list.md %}{% endif %}</p>
+  </header>
+
+  <div class="post-content" itemprop="articleBody">
+    {{ content }}
+  </div>
+
+</article>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_mentors/mentors.md
----------------------------------------------------------------------
diff --git a/website/_mentors/mentors.md b/website/_mentors/mentors.md
new file mode 100644
index 0000000..cdaca0a
--- /dev/null
+++ b/website/_mentors/mentors.md
@@ -0,0 +1,28 @@
+---
+group: "Mentors"
+description: "The following people are the mentors of this incubator project"
+members:
+  - name: Henry Saputra
+    apache_id: hsaputra
+    email: hsaputra [at] apache [dot] org
+    organization:
+    roles: PPMC, Mentor
+    time_zone: -8
+  - name: Flavio Junqueira
+    apache_id: fpj
+    email: fpj [at] apache [dot] org
+    organization: 
+    roles: PPMC, Mentor
+    time_zone: +0
+  - name: Chris Nauroth
+    apache_id: cnauroth
+    email: cnauroth [at] apache [dot] org
+    organization:
+    roles: PPMC, Mentor
+    time_zone:
+    apache_id: hsaputra
+    email: hsaputra [at] apache [dot] org
+    organization:
+    roles: PPMC, Mentor
+    time_zone: -8
+---

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/_base.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_base.scss b/website/_sass/_base.scss
new file mode 100644
index 0000000..0883c3c
--- /dev/null
+++ b/website/_sass/_base.scss
@@ -0,0 +1,206 @@
+/**
+ * Reset some basic elements
+ */
+body, h1, h2, h3, h4, h5, h6,
+p, blockquote, pre, hr,
+dl, dd, ol, ul, figure {
+    margin: 0;
+    padding: 0;
+}
+
+
+
+/**
+ * Basic styling
+ */
+body {
+    font: $base-font-weight #{$base-font-size}/#{$base-line-height} $base-font-family;
+    color: $text-color;
+    background-color: $background-color;
+    -webkit-text-size-adjust: 100%;
+    -webkit-font-feature-settings: "kern" 1;
+    -moz-font-feature-settings: "kern" 1;
+    -o-font-feature-settings: "kern" 1;
+    font-feature-settings: "kern" 1;
+    font-kerning: normal;
+}
+
+
+
+/**
+ * Set `margin-bottom` to maintain vertical rhythm
+ */
+h1, h2, h3, h4, h5, h6,
+p, blockquote, pre,
+ul, ol, dl, figure,
+%vertical-rhythm {
+    margin-bottom: $spacing-unit / 2;
+}
+
+
+
+/**
+ * Images
+ */
+img {
+    max-width: 100%;
+    vertical-align: middle;
+}
+
+
+
+/**
+ * Figures
+ */
+figure > img {
+    display: block;
+}
+
+figcaption {
+    font-size: $small-font-size;
+}
+
+
+
+/**
+ * Lists
+ */
+ul, ol {
+    margin-left: $spacing-unit;
+}
+
+li {
+    > ul,
+    > ol {
+         margin-bottom: 0;
+    }
+}
+
+
+
+/**
+ * Headings
+ */
+h1, h2, h3, h4, h5, h6 {
+    font-weight: $base-font-weight;
+}
+
+
+
+/**
+ * Links
+ */
+a {
+    color: $brand-color;
+    text-decoration: none;
+
+    &:visited {
+        color: darken($brand-color, 15%);
+    }
+
+    &:hover {
+        color: $text-color;
+        text-decoration: underline;
+    }
+}
+
+
+
+/**
+ * Blockquotes
+ */
+blockquote {
+    color: $grey-color;
+    border-left: 4px solid $grey-color-light;
+    padding-left: $spacing-unit / 2;
+    font-size: 18px;
+    letter-spacing: -1px;
+    font-style: italic;
+
+    > :last-child {
+        margin-bottom: 0;
+    }
+}
+
+
+
+/**
+ * Code formatting
+ */
+pre,
+code {
+    font-size: 15px;
+    border: 1px solid $grey-color-light;
+    border-radius: 3px;
+    background-color: #eef;
+}
+
+code {
+    padding: 1px 5px;
+}
+
+pre {
+    padding: 8px 12px;
+    overflow-x: auto;
+
+    > code {
+        border: 0;
+        padding-right: 0;
+        padding-left: 0;
+    }
+}
+
+
+
+/**
+ * Wrapper
+ */
+.wrapper {
+    max-width: -webkit-calc(#{$content-width} - (#{$spacing-unit} * 2));
+    max-width:         calc(#{$content-width} - (#{$spacing-unit} * 2));
+    margin-right: auto;
+    margin-left: auto;
+    padding-right: $spacing-unit;
+    padding-left: $spacing-unit;
+    @extend %clearfix;
+
+    @include media-query($on-laptop) {
+        max-width: -webkit-calc(#{$content-width} - (#{$spacing-unit}));
+        max-width:         calc(#{$content-width} - (#{$spacing-unit}));
+        padding-right: $spacing-unit / 2;
+        padding-left: $spacing-unit / 2;
+    }
+}
+
+
+
+/**
+ * Clearfix
+ */
+%clearfix {
+
+    &:after {
+        content: "";
+        display: table;
+        clear: both;
+    }
+}
+
+
+
+/**
+ * Icons
+ */
+.icon {
+
+    > svg {
+        display: inline-block;
+        width: 16px;
+        height: 16px;
+        vertical-align: middle;
+
+        path {
+            fill: $grey-color;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/_bootstrap.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_bootstrap.scss b/website/_sass/_bootstrap.scss
new file mode 100755
index 0000000..c773c8c
--- /dev/null
+++ b/website/_sass/_bootstrap.scss
@@ -0,0 +1,56 @@
+/*!
+ * Bootstrap v3.3.6 (http://getbootstrap.com)
+ * Copyright 2011-2015 Twitter, Inc.
+ * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+ */
+
+// Core variables and mixins
+@import "bootstrap/variables";
+@import "bootstrap/mixins";
+
+// Reset and dependencies
+@import "bootstrap/normalize";
+@import "bootstrap/print";
+@import "bootstrap/glyphicons";
+
+// Core CSS
+@import "bootstrap/scaffolding";
+@import "bootstrap/type";
+@import "bootstrap/code";
+@import "bootstrap/grid";
+@import "bootstrap/tables";
+@import "bootstrap/forms";
+@import "bootstrap/buttons";
+
+// Components
+@import "bootstrap/component-animations";
+@import "bootstrap/dropdowns";
+@import "bootstrap/button-groups";
+@import "bootstrap/input-groups";
+@import "bootstrap/navs";
+@import "bootstrap/navbar";
+@import "bootstrap/breadcrumbs";
+@import "bootstrap/pagination";
+@import "bootstrap/pager";
+@import "bootstrap/labels";
+@import "bootstrap/badges";
+@import "bootstrap/jumbotron";
+@import "bootstrap/thumbnails";
+@import "bootstrap/alerts";
+@import "bootstrap/progress-bars";
+@import "bootstrap/media";
+@import "bootstrap/list-group";
+@import "bootstrap/panels";
+@import "bootstrap/responsive-embed";
+@import "bootstrap/wells";
+@import "bootstrap/close";
+
+// Components w/ JavaScript
+@import "bootstrap/modals";
+@import "bootstrap/tooltip";
+@import "bootstrap/popovers";
+@import "bootstrap/carousel";
+
+// Utility classes
+@import "bootstrap/utilities";
+@import "bootstrap/responsive-utilities";

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/_layout.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_layout.scss b/website/_sass/_layout.scss
new file mode 100644
index 0000000..9cbfdde
--- /dev/null
+++ b/website/_sass/_layout.scss
@@ -0,0 +1,242 @@
+/**
+ * Site header
+ */
+.site-header {
+    border-top: 5px solid $grey-color-dark;
+    border-bottom: 1px solid $grey-color-light;
+    min-height: 56px;
+
+    // Positioning context for the mobile navigation icon
+    position: relative;
+}
+
+.site-title {
+    font-size: 26px;
+    font-weight: 300;
+    line-height: 56px;
+    letter-spacing: -1px;
+    margin-bottom: 0;
+    float: left;
+
+    &,
+    &:visited {
+        color: $grey-color-dark;
+    }
+}
+
+.site-nav {
+    float: right;
+    line-height: 56px;
+
+    .menu-icon {
+        display: none;
+    }
+
+    .page-link {
+        color: $text-color;
+        line-height: $base-line-height;
+
+        // Gaps between nav items, but not on the last one
+        &:not(:last-child) {
+            margin-right: 20px;
+        }
+    }
+
+    @include media-query($on-palm) {
+        position: absolute;
+        top: 9px;
+        right: $spacing-unit / 2;
+        background-color: $background-color;
+        border: 1px solid $grey-color-light;
+        border-radius: 5px;
+        text-align: right;
+
+        .menu-icon {
+            display: block;
+            float: right;
+            width: 36px;
+            height: 26px;
+            line-height: 0;
+            padding-top: 10px;
+            text-align: center;
+
+            > svg {
+                width: 18px;
+                height: 15px;
+
+                path {
+                    fill: $grey-color-dark;
+                }
+            }
+        }
+
+        .trigger {
+            clear: both;
+            display: none;
+        }
+
+        &:hover .trigger {
+            display: block;
+            padding-bottom: 5px;
+        }
+
+        .page-link {
+            display: block;
+            padding: 5px 10px;
+
+            &:not(:last-child) {
+                margin-right: 0;
+            }
+            margin-left: 20px;
+        }
+    }
+}
+
+
+
+/**
+ * Site footer
+ */
+.site-footer {
+    border-top: 1px solid $grey-color-light;
+    padding: $spacing-unit 0;
+}
+
+.footer-heading {
+    font-size: 18px;
+    margin-bottom: $spacing-unit / 2;
+}
+
+.contact-list,
+.social-media-list {
+    list-style: none;
+    margin-left: 0;
+}
+
+.footer-col-wrapper {
+    font-size: 15px;
+    color: $grey-color;
+    margin-left: -$spacing-unit / 2;
+    @extend %clearfix;
+}
+
+.footer-col {
+    float: left;
+    margin-bottom: $spacing-unit / 2;
+    padding-left: $spacing-unit / 2;
+}
+
+.footer-col-1 {
+    width: -webkit-calc(35% - (#{$spacing-unit} / 2));
+    width:         calc(35% - (#{$spacing-unit} / 2));
+}
+
+.footer-col-2 {
+    width: -webkit-calc(20% - (#{$spacing-unit} / 2));
+    width:         calc(20% - (#{$spacing-unit} / 2));
+}
+
+.footer-col-3 {
+    width: -webkit-calc(45% - (#{$spacing-unit} / 2));
+    width:         calc(45% - (#{$spacing-unit} / 2));
+}
+
+@include media-query($on-laptop) {
+    .footer-col-1,
+    .footer-col-2 {
+        width: -webkit-calc(50% - (#{$spacing-unit} / 2));
+        width:         calc(50% - (#{$spacing-unit} / 2));
+    }
+
+    .footer-col-3 {
+        width: -webkit-calc(100% - (#{$spacing-unit} / 2));
+        width:         calc(100% - (#{$spacing-unit} / 2));
+    }
+}
+
+@include media-query($on-palm) {
+    .footer-col {
+        float: none;
+        width: -webkit-calc(100% - (#{$spacing-unit} / 2));
+        width:         calc(100% - (#{$spacing-unit} / 2));
+    }
+}
+
+
+
+/**
+ * Page content
+ */
+.page-content {
+    padding: $spacing-unit 0;
+}
+
+.page-heading {
+    font-size: 20px;
+}
+
+.post-list {
+    margin-left: 0;
+    list-style: none;
+
+    > li {
+        margin-bottom: $spacing-unit;
+    }
+}
+
+.post-meta {
+    font-size: $small-font-size;
+    color: $grey-color;
+}
+
+.post-link {
+    display: block;
+    font-size: 24px;
+}
+
+
+
+/**
+ * Posts
+ */
+.post-header {
+    margin-bottom: $spacing-unit;
+}
+
+.post-title {
+    font-size: 42px;
+    letter-spacing: -1px;
+    line-height: 1;
+
+    @include media-query($on-laptop) {
+        font-size: 36px;
+    }
+}
+
+.post-content {
+    margin-bottom: $spacing-unit;
+
+    h2 {
+        font-size: 32px;
+
+        @include media-query($on-laptop) {
+            font-size: 28px;
+        }
+    }
+
+    h3 {
+        font-size: 26px;
+
+        @include media-query($on-laptop) {
+            font-size: 22px;
+        }
+    }
+
+    h4 {
+        font-size: 20px;
+
+        @include media-query($on-laptop) {
+            font-size: 18px;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/_syntax-highlighting.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_syntax-highlighting.scss b/website/_sass/_syntax-highlighting.scss
new file mode 100644
index 0000000..8fac597
--- /dev/null
+++ b/website/_sass/_syntax-highlighting.scss
@@ -0,0 +1,71 @@
+/**
+ * Syntax highlighting styles
+ */
+.highlight {
+    background: #fff;
+    @extend %vertical-rhythm;
+
+    .highlighter-rouge & {
+      background: #eef;
+    }
+
+    .c     { color: #998; font-style: italic } // Comment
+    .err   { color: #a61717; background-color: #e3d2d2 } // Error
+    .k     { font-weight: bold } // Keyword
+    .o     { font-weight: bold } // Operator
+    .cm    { color: #998; font-style: italic } // Comment.Multiline
+    .cp    { color: #999; font-weight: bold } // Comment.Preproc
+    .c1    { color: #998; font-style: italic } // Comment.Single
+    .cs    { color: #999; font-weight: bold; font-style: italic } // Comment.Special
+    .gd    { color: #000; background-color: #fdd } // Generic.Deleted
+    .gd .x { color: #000; background-color: #faa } // Generic.Deleted.Specific
+    .ge    { font-style: italic } // Generic.Emph
+    .gr    { color: #a00 } // Generic.Error
+    .gh    { color: #999 } // Generic.Heading
+    .gi    { color: #000; background-color: #dfd } // Generic.Inserted
+    .gi .x { color: #000; background-color: #afa } // Generic.Inserted.Specific
+    .go    { color: #888 } // Generic.Output
+    .gp    { color: #555 } // Generic.Prompt
+    .gs    { font-weight: bold } // Generic.Strong
+    .gu    { color: #aaa } // Generic.Subheading
+    .gt    { color: #a00 } // Generic.Traceback
+    .kc    { font-weight: bold } // Keyword.Constant
+    .kd    { font-weight: bold } // Keyword.Declaration
+    .kp    { font-weight: bold } // Keyword.Pseudo
+    .kr    { font-weight: bold } // Keyword.Reserved
+    .kt    { color: #458; font-weight: bold } // Keyword.Type
+    .m     { color: #099 } // Literal.Number
+    .s     { color: #d14 } // Literal.String
+    .na    { color: #008080 } // Name.Attribute
+    .nb    { color: #0086B3 } // Name.Builtin
+    .nc    { color: #458; font-weight: bold } // Name.Class
+    .no    { color: #008080 } // Name.Constant
+    .ni    { color: #800080 } // Name.Entity
+    .ne    { color: #900; font-weight: bold } // Name.Exception
+    .nf    { color: #900; font-weight: bold } // Name.Function
+    .nn    { color: #555 } // Name.Namespace
+    .nt    { color: #000080 } // Name.Tag
+    .nv    { color: #008080 } // Name.Variable
+    .ow    { font-weight: bold } // Operator.Word
+    .w     { color: #bbb } // Text.Whitespace
+    .mf    { color: #099 } // Literal.Number.Float
+    .mh    { color: #099 } // Literal.Number.Hex
+    .mi    { color: #099 } // Literal.Number.Integer
+    .mo    { color: #099 } // Literal.Number.Oct
+    .sb    { color: #d14 } // Literal.String.Backtick
+    .sc    { color: #d14 } // Literal.String.Char
+    .sd    { color: #d14 } // Literal.String.Doc
+    .s2    { color: #d14 } // Literal.String.Double
+    .se    { color: #d14 } // Literal.String.Escape
+    .sh    { color: #d14 } // Literal.String.Heredoc
+    .si    { color: #d14 } // Literal.String.Interpol
+    .sx    { color: #d14 } // Literal.String.Other
+    .sr    { color: #009926 } // Literal.String.Regex
+    .s1    { color: #d14 } // Literal.String.Single
+    .ss    { color: #990073 } // Literal.String.Symbol
+    .bp    { color: #999 } // Name.Builtin.Pseudo
+    .vc    { color: #008080 } // Name.Variable.Class
+    .vg    { color: #008080 } // Name.Variable.Global
+    .vi    { color: #008080 } // Name.Variable.Instance
+    .il    { color: #099 } // Literal.Number.Integer.Long
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_alerts.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_alerts.scss b/website/_sass/bootstrap/_alerts.scss
new file mode 100755
index 0000000..7d1e1fd
--- /dev/null
+++ b/website/_sass/bootstrap/_alerts.scss
@@ -0,0 +1,73 @@
+//
+// Alerts
+// --------------------------------------------------
+
+
+// Base styles
+// -------------------------
+
+.alert {
+  padding: $alert-padding;
+  margin-bottom: $line-height-computed;
+  border: 1px solid transparent;
+  border-radius: $alert-border-radius;
+
+  // Headings for larger alerts
+  h4 {
+    margin-top: 0;
+    // Specified for the h4 to prevent conflicts of changing $headings-color
+    color: inherit;
+  }
+
+  // Provide class for links that match alerts
+  .alert-link {
+    font-weight: $alert-link-font-weight;
+  }
+
+  // Improve alignment and spacing of inner content
+  > p,
+  > ul {
+    margin-bottom: 0;
+  }
+
+  > p + p {
+    margin-top: 5px;
+  }
+}
+
+// Dismissible alerts
+//
+// Expand the right padding and account for the close button's positioning.
+
+.alert-dismissable, // The misspelled .alert-dismissable was deprecated in 3.2.0.
+.alert-dismissible {
+  padding-right: ($alert-padding + 20);
+
+  // Adjust close link position
+  .close {
+    position: relative;
+    top: -2px;
+    right: -21px;
+    color: inherit;
+  }
+}
+
+// Alternate styles
+//
+// Generate contextual modifier classes for colorizing the alert.
+
+.alert-success {
+  @include alert-variant($alert-success-bg, $alert-success-border, $alert-success-text);
+}
+
+.alert-info {
+  @include alert-variant($alert-info-bg, $alert-info-border, $alert-info-text);
+}
+
+.alert-warning {
+  @include alert-variant($alert-warning-bg, $alert-warning-border, $alert-warning-text);
+}
+
+.alert-danger {
+  @include alert-variant($alert-danger-bg, $alert-danger-border, $alert-danger-text);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_badges.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_badges.scss b/website/_sass/bootstrap/_badges.scss
new file mode 100755
index 0000000..70002e0
--- /dev/null
+++ b/website/_sass/bootstrap/_badges.scss
@@ -0,0 +1,68 @@
+//
+// Badges
+// --------------------------------------------------
+
+
+// Base class
+.badge {
+  display: inline-block;
+  min-width: 10px;
+  padding: 3px 7px;
+  font-size: $font-size-small;
+  font-weight: $badge-font-weight;
+  color: $badge-color;
+  line-height: $badge-line-height;
+  vertical-align: middle;
+  white-space: nowrap;
+  text-align: center;
+  background-color: $badge-bg;
+  border-radius: $badge-border-radius;
+
+  // Empty badges collapse automatically (not available in IE8)
+  &:empty {
+    display: none;
+  }
+
+  // Quick fix for badges in buttons
+  .btn & {
+    position: relative;
+    top: -1px;
+  }
+
+  .btn-xs &,
+  .btn-group-xs > .btn & {
+    top: 0;
+    padding: 1px 5px;
+  }
+
+  // [converter] extracted a& to a.badge
+
+  // Account for badges in navs
+  .list-group-item.active > &,
+  .nav-pills > .active > a > & {
+    color: $badge-active-color;
+    background-color: $badge-active-bg;
+  }
+
+  .list-group-item > & {
+    float: right;
+  }
+
+  .list-group-item > & + & {
+    margin-right: 5px;
+  }
+
+  .nav-pills > li > a > & {
+    margin-left: 3px;
+  }
+}
+
+// Hover state, but only for links
+a.badge {
+  &:hover,
+  &:focus {
+    color: $badge-link-hover-color;
+    text-decoration: none;
+    cursor: pointer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_breadcrumbs.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_breadcrumbs.scss b/website/_sass/bootstrap/_breadcrumbs.scss
new file mode 100755
index 0000000..b61f0c7
--- /dev/null
+++ b/website/_sass/bootstrap/_breadcrumbs.scss
@@ -0,0 +1,28 @@
+//
+// Breadcrumbs
+// --------------------------------------------------
+
+
+.breadcrumb {
+  padding: $breadcrumb-padding-vertical $breadcrumb-padding-horizontal;
+  margin-bottom: $line-height-computed;
+  list-style: none;
+  background-color: $breadcrumb-bg;
+  border-radius: $border-radius-base;
+
+  > li {
+    display: inline-block;
+
+    + li:before {
+      // [converter] Workaround for https://github.com/sass/libsass/issues/1115
+      $nbsp: "\00a0";
+      content: "#{$breadcrumb-separator}#{$nbsp}"; // Unicode space added since inline-block means non-collapsing white-space
+      padding: 0 5px;
+      color: $breadcrumb-color;
+    }
+  }
+
+  > .active {
+    color: $breadcrumb-active-color;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_button-groups.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_button-groups.scss b/website/_sass/bootstrap/_button-groups.scss
new file mode 100755
index 0000000..baaacc4
--- /dev/null
+++ b/website/_sass/bootstrap/_button-groups.scss
@@ -0,0 +1,244 @@
+//
+// Button groups
+// --------------------------------------------------
+
+// Make the div behave like a button
+.btn-group,
+.btn-group-vertical {
+  position: relative;
+  display: inline-block;
+  vertical-align: middle; // match .btn alignment given font-size hack above
+  > .btn {
+    position: relative;
+    float: left;
+    // Bring the "active" button to the front
+    &:hover,
+    &:focus,
+    &:active,
+    &.active {
+      z-index: 2;
+    }
+  }
+}
+
+// Prevent double borders when buttons are next to each other
+.btn-group {
+  .btn + .btn,
+  .btn + .btn-group,
+  .btn-group + .btn,
+  .btn-group + .btn-group {
+    margin-left: -1px;
+  }
+}
+
+// Optional: Group multiple button groups together for a toolbar
+.btn-toolbar {
+  margin-left: -5px; // Offset the first child's margin
+  @include clearfix;
+
+  .btn,
+  .btn-group,
+  .input-group {
+    float: left;
+  }
+  > .btn,
+  > .btn-group,
+  > .input-group {
+    margin-left: 5px;
+  }
+}
+
+.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {
+  border-radius: 0;
+}
+
+// Set corners individual because sometimes a single button can be in a .btn-group and we need :first-child and :last-child to both match
+.btn-group > .btn:first-child {
+  margin-left: 0;
+  &:not(:last-child):not(.dropdown-toggle) {
+    @include border-right-radius(0);
+  }
+}
+// Need .dropdown-toggle since :last-child doesn't apply given a .dropdown-menu immediately after it
+.btn-group > .btn:last-child:not(:first-child),
+.btn-group > .dropdown-toggle:not(:first-child) {
+  @include border-left-radius(0);
+}
+
+// Custom edits for including btn-groups within btn-groups (useful for including dropdown buttons within a btn-group)
+.btn-group > .btn-group {
+  float: left;
+}
+.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group > .btn-group:first-child:not(:last-child) {
+  > .btn:last-child,
+  > .dropdown-toggle {
+    @include border-right-radius(0);
+  }
+}
+.btn-group > .btn-group:last-child:not(:first-child) > .btn:first-child {
+  @include border-left-radius(0);
+}
+
+// On active and open, don't show outline
+.btn-group .dropdown-toggle:active,
+.btn-group.open .dropdown-toggle {
+  outline: 0;
+}
+
+
+// Sizing
+//
+// Remix the default button sizing classes into new ones for easier manipulation.
+
+.btn-group-xs > .btn { @extend .btn-xs; }
+.btn-group-sm > .btn { @extend .btn-sm; }
+.btn-group-lg > .btn { @extend .btn-lg; }
+
+
+// Split button dropdowns
+// ----------------------
+
+// Give the line between buttons some depth
+.btn-group > .btn + .dropdown-toggle {
+  padding-left: 8px;
+  padding-right: 8px;
+}
+.btn-group > .btn-lg + .dropdown-toggle {
+  padding-left: 12px;
+  padding-right: 12px;
+}
+
+// The clickable button for toggling the menu
+// Remove the gradient and set the same inset shadow as the :active state
+.btn-group.open .dropdown-toggle {
+  @include box-shadow(inset 0 3px 5px rgba(0,0,0,.125));
+
+  // Show no shadow for `.btn-link` since it has no other button styles.
+  &.btn-link {
+    @include box-shadow(none);
+  }
+}
+
+
+// Reposition the caret
+.btn .caret {
+  margin-left: 0;
+}
+// Carets in other button sizes
+.btn-lg .caret {
+  border-width: $caret-width-large $caret-width-large 0;
+  border-bottom-width: 0;
+}
+// Upside down carets for .dropup
+.dropup .btn-lg .caret {
+  border-width: 0 $caret-width-large $caret-width-large;
+}
+
+
+// Vertical button groups
+// ----------------------
+
+.btn-group-vertical {
+  > .btn,
+  > .btn-group,
+  > .btn-group > .btn {
+    display: block;
+    float: none;
+    width: 100%;
+    max-width: 100%;
+  }
+
+  // Clear floats so dropdown menus can be properly placed
+  > .btn-group {
+    @include clearfix;
+    > .btn {
+      float: none;
+    }
+  }
+
+  > .btn + .btn,
+  > .btn + .btn-group,
+  > .btn-group + .btn,
+  > .btn-group + .btn-group {
+    margin-top: -1px;
+    margin-left: 0;
+  }
+}
+
+.btn-group-vertical > .btn {
+  &:not(:first-child):not(:last-child) {
+    border-radius: 0;
+  }
+  &:first-child:not(:last-child) {
+    @include border-top-radius($btn-border-radius-base);
+    @include border-bottom-radius(0);
+  }
+  &:last-child:not(:first-child) {
+    @include border-top-radius(0);
+    @include border-bottom-radius($btn-border-radius-base);
+  }
+}
+.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn {
+  border-radius: 0;
+}
+.btn-group-vertical > .btn-group:first-child:not(:last-child) {
+  > .btn:last-child,
+  > .dropdown-toggle {
+    @include border-bottom-radius(0);
+  }
+}
+.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child {
+  @include border-top-radius(0);
+}
+
+
+// Justified button groups
+// ----------------------
+
+.btn-group-justified {
+  display: table;
+  width: 100%;
+  table-layout: fixed;
+  border-collapse: separate;
+  > .btn,
+  > .btn-group {
+    float: none;
+    display: table-cell;
+    width: 1%;
+  }
+  > .btn-group .btn {
+    width: 100%;
+  }
+
+  > .btn-group .dropdown-menu {
+    left: auto;
+  }
+}
+
+
+// Checkbox and radio options
+//
+// In order to support the browser's form validation feedback, powered by the
+// `required` attribute, we have to "hide" the inputs via `clip`. We cannot use
+// `display: none;` or `visibility: hidden;` as that also hides the popover.
+// Simply visually hiding the inputs via `opacity` would leave them clickable in
+// certain cases which is prevented by using `clip` and `pointer-events`.
+// This way, we ensure a DOM element is visible to position the popover from.
+//
+// See https://github.com/twbs/bootstrap/pull/12794 and
+// https://github.com/twbs/bootstrap/pull/14559 for more information.
+
+[data-toggle="buttons"] {
+  > .btn,
+  > .btn-group > .btn {
+    input[type="radio"],
+    input[type="checkbox"] {
+      position: absolute;
+      clip: rect(0,0,0,0);
+      pointer-events: none;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_buttons.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_buttons.scss b/website/_sass/bootstrap/_buttons.scss
new file mode 100755
index 0000000..6452b70
--- /dev/null
+++ b/website/_sass/bootstrap/_buttons.scss
@@ -0,0 +1,168 @@
+//
+// Buttons
+// --------------------------------------------------
+
+
+// Base styles
+// --------------------------------------------------
+
+.btn {
+  display: inline-block;
+  margin-bottom: 0; // For input.btn
+  font-weight: $btn-font-weight;
+  text-align: center;
+  vertical-align: middle;
+  touch-action: manipulation;
+  cursor: pointer;
+  background-image: none; // Reset unusual Firefox-on-Android default style; see https://github.com/necolas/normalize.css/issues/214
+  border: 1px solid transparent;
+  white-space: nowrap;
+  @include button-size($padding-base-vertical, $padding-base-horizontal, $font-size-base, $line-height-base, $btn-border-radius-base);
+  @include user-select(none);
+
+  &,
+  &:active,
+  &.active {
+    &:focus,
+    &.focus {
+      @include tab-focus;
+    }
+  }
+
+  &:hover,
+  &:focus,
+  &.focus {
+    color: $btn-default-color;
+    text-decoration: none;
+  }
+
+  &:active,
+  &.active {
+    outline: 0;
+    background-image: none;
+    @include box-shadow(inset 0 3px 5px rgba(0,0,0,.125));
+  }
+
+  &.disabled,
+  &[disabled],
+  fieldset[disabled] & {
+    cursor: $cursor-disabled;
+    @include opacity(.65);
+    @include box-shadow(none);
+  }
+
+  // [converter] extracted a& to a.btn
+}
+
+a.btn {
+  &.disabled,
+  fieldset[disabled] & {
+    pointer-events: none; // Future-proof disabling of clicks on `<a>` elements
+  }
+}
+
+
+// Alternate buttons
+// --------------------------------------------------
+
+.btn-default {
+  @include button-variant($btn-default-color, $btn-default-bg, $btn-default-border);
+}
+.btn-primary {
+  @include button-variant($btn-primary-color, $btn-primary-bg, $btn-primary-border);
+}
+// Success appears as green
+.btn-success {
+  @include button-variant($btn-success-color, $btn-success-bg, $btn-success-border);
+}
+// Info appears as blue-green
+.btn-info {
+  @include button-variant($btn-info-color, $btn-info-bg, $btn-info-border);
+}
+// Warning appears as orange
+.btn-warning {
+  @include button-variant($btn-warning-color, $btn-warning-bg, $btn-warning-border);
+}
+// Danger and error appear as red
+.btn-danger {
+  @include button-variant($btn-danger-color, $btn-danger-bg, $btn-danger-border);
+}
+
+
+// Link buttons
+// -------------------------
+
+// Make a button look and behave like a link
+.btn-link {
+  color: $link-color;
+  font-weight: normal;
+  border-radius: 0;
+
+  &,
+  &:active,
+  &.active,
+  &[disabled],
+  fieldset[disabled] & {
+    background-color: transparent;
+    @include box-shadow(none);
+  }
+  &,
+  &:hover,
+  &:focus,
+  &:active {
+    border-color: transparent;
+  }
+  &:hover,
+  &:focus {
+    color: $link-hover-color;
+    text-decoration: $link-hover-decoration;
+    background-color: transparent;
+  }
+  &[disabled],
+  fieldset[disabled] & {
+    &:hover,
+    &:focus {
+      color: $btn-link-disabled-color;
+      text-decoration: none;
+    }
+  }
+}
+
+
+// Button Sizes
+// --------------------------------------------------
+
+.btn-lg {
+  // line-height: ensure even-numbered height of button next to large input
+  @include button-size($padding-large-vertical, $padding-large-horizontal, $font-size-large, $line-height-large, $btn-border-radius-large);
+}
+.btn-sm {
+  // line-height: ensure proper height of button next to small input
+  @include button-size($padding-small-vertical, $padding-small-horizontal, $font-size-small, $line-height-small, $btn-border-radius-small);
+}
+.btn-xs {
+  @include button-size($padding-xs-vertical, $padding-xs-horizontal, $font-size-small, $line-height-small, $btn-border-radius-small);
+}
+
+
+// Block button
+// --------------------------------------------------
+
+.btn-block {
+  display: block;
+  width: 100%;
+}
+
+// Vertically space out multiple block buttons
+.btn-block + .btn-block {
+  margin-top: 5px;
+}
+
+// Specificity overrides
+input[type="submit"],
+input[type="reset"],
+input[type="button"] {
+  &.btn-block {
+    width: 100%;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_carousel.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_carousel.scss b/website/_sass/bootstrap/_carousel.scss
new file mode 100755
index 0000000..753d881
--- /dev/null
+++ b/website/_sass/bootstrap/_carousel.scss
@@ -0,0 +1,270 @@
+//
+// Carousel
+// --------------------------------------------------
+
+
+// Wrapper for the slide container and indicators
+.carousel {
+  position: relative;
+}
+
+.carousel-inner {
+  position: relative;
+  overflow: hidden;
+  width: 100%;
+
+  > .item {
+    display: none;
+    position: relative;
+    @include transition(.6s ease-in-out left);
+
+    // Account for jankitude on images
+    > img,
+    > a > img {
+      @include img-responsive;
+      line-height: 1;
+    }
+
+    // WebKit CSS3 transforms for supported devices
+    @media all and (transform-3d), (-webkit-transform-3d) {
+      @include transition-transform(0.6s ease-in-out);
+      @include backface-visibility(hidden);
+      @include perspective(1000px);
+
+      &.next,
+      &.active.right {
+        @include translate3d(100%, 0, 0);
+        left: 0;
+      }
+      &.prev,
+      &.active.left {
+        @include translate3d(-100%, 0, 0);
+        left: 0;
+      }
+      &.next.left,
+      &.prev.right,
+      &.active {
+        @include translate3d(0, 0, 0);
+        left: 0;
+      }
+    }
+  }
+
+  > .active,
+  > .next,
+  > .prev {
+    display: block;
+  }
+
+  > .active {
+    left: 0;
+  }
+
+  > .next,
+  > .prev {
+    position: absolute;
+    top: 0;
+    width: 100%;
+  }
+
+  > .next {
+    left: 100%;
+  }
+  > .prev {
+    left: -100%;
+  }
+  > .next.left,
+  > .prev.right {
+    left: 0;
+  }
+
+  > .active.left {
+    left: -100%;
+  }
+  > .active.right {
+    left: 100%;
+  }
+
+}
+
+// Left/right controls for nav
+// ---------------------------
+
+.carousel-control {
+  position: absolute;
+  top: 0;
+  left: 0;
+  bottom: 0;
+  width: $carousel-control-width;
+  @include opacity($carousel-control-opacity);
+  font-size: $carousel-control-font-size;
+  color: $carousel-control-color;
+  text-align: center;
+  text-shadow: $carousel-text-shadow;
+  background-color: rgba(0, 0, 0, 0); // Fix IE9 click-thru bug
+  // We can't have this transition here because WebKit cancels the carousel
+  // animation if you trip this while in the middle of another animation.
+
+  // Set gradients for backgrounds
+  &.left {
+    @include gradient-horizontal($start-color: rgba(0,0,0,.5), $end-color: rgba(0,0,0,.0001));
+  }
+  &.right {
+    left: auto;
+    right: 0;
+    @include gradient-horizontal($start-color: rgba(0,0,0,.0001), $end-color: rgba(0,0,0,.5));
+  }
+
+  // Hover/focus state
+  &:hover,
+  &:focus {
+    outline: 0;
+    color: $carousel-control-color;
+    text-decoration: none;
+    @include opacity(.9);
+  }
+
+  // Toggles
+  .icon-prev,
+  .icon-next,
+  .glyphicon-chevron-left,
+  .glyphicon-chevron-right {
+    position: absolute;
+    top: 50%;
+    margin-top: -10px;
+    z-index: 5;
+    display: inline-block;
+  }
+  .icon-prev,
+  .glyphicon-chevron-left {
+    left: 50%;
+    margin-left: -10px;
+  }
+  .icon-next,
+  .glyphicon-chevron-right {
+    right: 50%;
+    margin-right: -10px;
+  }
+  .icon-prev,
+  .icon-next {
+    width:  20px;
+    height: 20px;
+    line-height: 1;
+    font-family: serif;
+  }
+
+
+  .icon-prev {
+    &:before {
+      content: '\2039';// SINGLE LEFT-POINTING ANGLE QUOTATION MARK (U+2039)
+    }
+  }
+  .icon-next {
+    &:before {
+      content: '\203a';// SINGLE RIGHT-POINTING ANGLE QUOTATION MARK (U+203A)
+    }
+  }
+}
+
+// Optional indicator pips
+//
+// Add an unordered list with the following class and add a list item for each
+// slide your carousel holds.
+
+.carousel-indicators {
+  position: absolute;
+  bottom: 10px;
+  left: 50%;
+  z-index: 15;
+  width: 60%;
+  margin-left: -30%;
+  padding-left: 0;
+  list-style: none;
+  text-align: center;
+
+  li {
+    display: inline-block;
+    width:  10px;
+    height: 10px;
+    margin: 1px;
+    text-indent: -999px;
+    border: 1px solid $carousel-indicator-border-color;
+    border-radius: 10px;
+    cursor: pointer;
+
+    // IE8-9 hack for event handling
+    //
+    // Internet Explorer 8-9 does not support clicks on elements without a set
+    // `background-color`. We cannot use `filter` since that's not viewed as a
+    // background color by the browser. Thus, a hack is needed.
+    // See https://developer.mozilla.org/en-US/docs/Web/Events/click#Internet_Explorer
+    //
+    // For IE8, we set solid black as it doesn't support `rgba()`. For IE9, we
+    // set alpha transparency for the best results possible.
+    background-color: #000 \9; // IE8
+    background-color: rgba(0,0,0,0); // IE9
+  }
+  .active {
+    margin: 0;
+    width:  12px;
+    height: 12px;
+    background-color: $carousel-indicator-active-bg;
+  }
+}
+
+// Optional captions
+// -----------------------------
+// Hidden by default for smaller viewports
+.carousel-caption {
+  position: absolute;
+  left: 15%;
+  right: 15%;
+  bottom: 20px;
+  z-index: 10;
+  padding-top: 20px;
+  padding-bottom: 20px;
+  color: $carousel-caption-color;
+  text-align: center;
+  text-shadow: $carousel-text-shadow;
+  & .btn {
+    text-shadow: none; // No shadow for button elements in carousel-caption
+  }
+}
+
+
+// Scale up controls for tablets and up
+@media screen and (min-width: $screen-sm-min) {
+
+  // Scale up the controls a smidge
+  .carousel-control {
+    .glyphicon-chevron-left,
+    .glyphicon-chevron-right,
+    .icon-prev,
+    .icon-next {
+      width: ($carousel-control-font-size * 1.5);
+      height: ($carousel-control-font-size * 1.5);
+      margin-top: ($carousel-control-font-size / -2);
+      font-size: ($carousel-control-font-size * 1.5);
+    }
+    .glyphicon-chevron-left,
+    .icon-prev {
+      margin-left: ($carousel-control-font-size / -2);
+    }
+    .glyphicon-chevron-right,
+    .icon-next {
+      margin-right: ($carousel-control-font-size / -2);
+    }
+  }
+
+  // Show and left align the captions
+  .carousel-caption {
+    left: 20%;
+    right: 20%;
+    padding-bottom: 30px;
+  }
+
+  // Move up the indicators
+  .carousel-indicators {
+    bottom: 20px;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b169db04/website/_sass/bootstrap/_close.scss
----------------------------------------------------------------------
diff --git a/website/_sass/bootstrap/_close.scss b/website/_sass/bootstrap/_close.scss
new file mode 100755
index 0000000..3b74d8a
--- /dev/null
+++ b/website/_sass/bootstrap/_close.scss
@@ -0,0 +1,36 @@
+//
+// Close icons
+// --------------------------------------------------
+
+
+.close {
+  float: right;
+  font-size: ($font-size-base * 1.5);
+  font-weight: $close-font-weight;
+  line-height: 1;
+  color: $close-color;
+  text-shadow: $close-text-shadow;
+  @include opacity(.2);
+
+  &:hover,
+  &:focus {
+    color: $close-color;
+    text-decoration: none;
+    cursor: pointer;
+    @include opacity(.5);
+  }
+
+  // [converter] extracted button& to button.close
+}
+
+// Additional properties for button version
+// iOS requires the button element instead of an anchor tag.
+// If you want the anchor version, it requires `href="#"`.
+// See https://developer.mozilla.org/en-US/docs/Web/Events/click#Safari_Mobile
+button.close {
+  padding: 0;
+  cursor: pointer;
+  background: transparent;
+  border: 0;
+  -webkit-appearance: none;
+}