You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@fluo.apache.org by mw...@apache.org on 2017/06/13 16:45:23 UTC

[incubator-fluo-website] branch gh-pages updated: Adding 1.1.0 documentation to the website (#58)

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

mwalch pushed a commit to branch gh-pages
in repository https://gitbox.apache.org/repos/asf/incubator-fluo-website.git


The following commit(s) were added to refs/heads/gh-pages by this push:
     new 31beab2  Adding 1.1.0 documentation to the website (#58)
31beab2 is described below

commit 31beab2fc4d23c1b8b3e7ecea3726e64fd070987
Author: Mike Walch <mw...@apache.org>
AuthorDate: Tue Jun 13 12:45:21 2017 -0400

    Adding 1.1.0 documentation to the website (#58)
---
 _scripts/convert-fluo-docs.py                      |   4 +-
 docs/fluo/1.1.0-incubating/applications.md         | 209 ++++++++++++++++++
 docs/fluo/1.1.0-incubating/architecture.md         |  44 ++++
 docs/fluo/1.1.0-incubating/contributing.md         |  30 +++
 docs/fluo/1.1.0-incubating/grafana.md              |  83 +++++++
 docs/fluo/1.1.0-incubating/index.md                |  56 +++++
 docs/fluo/1.1.0-incubating/install.md              | 238 +++++++++++++++++++++
 docs/fluo/1.1.0-incubating/metrics.md              | 116 ++++++++++
 .../resources/fluo-architecture.png                | Bin 0 -> 61085 bytes
 docs/index.md                                      |   2 +
 10 files changed, 779 insertions(+), 3 deletions(-)

diff --git a/_scripts/convert-fluo-docs.py b/_scripts/convert-fluo-docs.py
index 1c1c828..8e593ac 100755
--- a/_scripts/convert-fluo-docs.py
+++ b/_scripts/convert-fluo-docs.py
@@ -14,10 +14,8 @@ output_dir = sys.argv[2]
 
 args = output_dir.rpartition("/docs")
 url_prefix = args[1] + args[2]
-print url_prefix
-print args
 release_ver = url_prefix.split("/")[3]
-github_prefix = "https://github.com/apache/fluo/blob/{0}/modules/".format(release_ver)
+github_prefix = "https://github.com/apache/fluo/blob/rel/fluo-{0}/modules/".format(release_ver)
 javadocs_prefix = "{{ site.fluo_api_static }}/" + release_ver + "/"
 resources_prefix = "/docs/fluo/{0}/resources/".format(release_ver)
 
diff --git a/docs/fluo/1.1.0-incubating/applications.md b/docs/fluo/1.1.0-incubating/applications.md
new file mode 100644
index 0000000..7da62e7
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/applications.md
@@ -0,0 +1,209 @@
+---
+layout: fluo-doc
+title:  Fluo Applications
+version:  1.1.0-incubating
+---
+Once you have Fluo installed and running on your cluster, you can now run Fluo applications which
+consist of clients and observers.
+
+For both clients and observers, you will need to include the following in your Maven pom:
+
+```xml
+<dependency>
+  <groupId>org.apache.fluo</groupId>
+  <artifactId>fluo-api</artifactId>
+  <version>1.x.x-incubating</version>
+</dependency>
+<dependency>
+  <groupId>org.apache.fluo</groupId>
+  <artifactId>fluo-core</artifactId>
+  <version>1.x.x-incubating</version>
+  <scope>runtime</scope>
+</dependency>
+```
+
+Fluo provides a classpath command to help users build a runtime classpath. This command along with
+the `hadoop jar` command is useful when writing scripts to run Fluo client code. These commands
+allow the scripts to use the versions of Hadoop, Accumulo, and Zookeeper installed on a cluster.
+
+## Creating a Fluo client
+
+To create a [FluoClient], you will need to provide it with a [FluoConfiguration] object that is
+configured to connect to your Fluo instance.
+
+If you have access to the [fluo.properties] file that was used to configure your Fluo instance, you
+can use it to build a [FluoConfiguration] object with all necessary properties which are all
+properties with the `fluo.client.*` prefix in [fluo.properties]:
+
+```java
+FluoConfiguration config = new FluoConfiguration(new File("fluo.properties"));
+```
+
+You can also create an empty [FluoConfiguration] object and set properties using Java:
+
+```java
+FluoConfiguration config = new FluoConfiguration();
+config.setAccumuloUser("user");
+config.setAccumuloPassword("pass");
+config.setAccumuloInstance("instance");
+```
+
+Once you have [FluoConfiguration] object, pass it to the `newClient()` method of [FluoFactory] to
+create a [FluoClient]:
+
+```java
+FluoClient client = FluoFactory.newClient(config)
+```
+
+It may help to reference the [API javadocs][API] while you are learning the Fluo API.
+
+## Running application code
+
+The `fluo exec <app name> <class> {arguments}` provides an easy way to execute application code. It
+will execute a class with a main method if a jar containing the class is placed in the lib directory
+of the application. When the class is run, Fluo classes and dependencies will be on the classpath.
+The `fluo exec` command can inject the applications configuration if the class is written in the
+following way. Defining the injection point is optional.
+
+```java
+import javax.inject.Inject;
+
+public class AppCommand {
+
+  //when run with fluo exec command, the applications configuration will be injected
+  @Inject
+  private static FluoConfiguration fluoConfig;
+
+  public static void main(String[] args) throws Exception {
+    try(FluoClient fluoClient = FluoFactory.newClient(fluoConfig)) {
+      //do stuff with Fluo
+    }
+  }
+}
+```
+
+## Creating a Fluo observer
+
+To create an observer, follow these steps:
+
+1.  Create one or more classes that extend [Observer] like the example below. Please use [slf4j] for
+    any logging in observers as [slf4j] supports multiple logging implementations. This is
+    necessary as Fluo applications have a hard requirement on [logback] when running in YARN.
+
+    ```java
+    public class InvertObserver implements Observer {
+
+      @Override
+      public void process(TransactionBase tx, Bytes row, Column col) throws Exception {
+        // read value
+        Bytes value = tx.get(row, col);
+        // invert row and value
+        tx.set(value, new Column("inv", "data"), row);
+      }
+    }
+    ```
+
+2.  Create a class that implements [ObserverProvider] like the example below.  The purpose of this
+    class is associate a set Observers with columns that trigger the observers.  The class can
+    register multiple observers.
+
+    ```java
+    class AppObserverProvider implements ObserverProvider {
+      @Override
+      public void provide(Registry or, Context ctx) {
+        //setup InvertObserver to be triggered when the column obs:data is modified
+        or.forColumn(new Column("obs", "data"), NotificationType.STRONG)
+          .useObserver(new InvertObserver());
+        
+        //Observer is a Functional interface.  So Obsevers can be written as lambdas.
+        or.forColumn(new Column("new","data"), NotificationType.WEAK)
+          .useObserver((tx,row,col) -> {
+             Bytes combined = combineNewAndOld(tx,row);
+             tx.set(row, new Column("current","data"), combined);
+           });
+      }
+    }
+    ```
+
+3.  Build a jar containing thses classes and include this jar in the `lib/` directory of your Fluo
+    application.
+4.  Configure your Fluo instance to use this observer provider by modifying the Observer section of
+    [fluo.properties].
+5.  Initialize Fluo.  During initialization Fluo will obtain the observed columns from the 
+    ObserverProvider and persist the columns in Zookeeper.  These columns persisted in Zookeeper
+    are used by transactions to know when to trigger observers.
+6.  Start your Fluo instance so that your Fluo workers load the new observer.
+
+## Application Configuration
+
+For configuring observers, fluo provides a simple mechanism to set and access application specific
+configuration.  See the javadoc on [FluoClient].getAppConfiguration() for more details.
+
+## Debugging Applications
+
+While monitoring [Fluo metrics][metrics] can detect problems (like too many transaction collisions)
+in a Fluo application, [metrics][metrics] may not provide enough information to debug the root cause
+of the problem. To help debug Fluo applications, low-level logging of transactions can be turned on
+by setting the following loggers to TRACE:
+
+| Logger               | Level | Information                                                                                        |
+|----------------------|-------|----------------------------------------------------------------------------------------------------|
+| fluo.tx            | TRACE | Provides detailed information about what transactions read and wrote                               |
+| fluo.tx.summary    | TRACE | Provides a one line summary about each transaction executed                                        |
+| fluo.tx.collisions | TRACE | Provides details about what data was involved When a transaction collides with another transaction |
+| fluo.tx.scan       | TRACE | Provides logging for each cell read by a scan.  Scan summary logged at `fluo.tx` level.  This allows suppression of `fluo.tx.scan` while still seeing summary. |
+
+Below is an example log after setting `fluo.tx` to TRACE. The number following `txid: ` is the
+transactions start timestamp from the Oracle.
+
+```
+2015-02-11 18:24:05,341 [fluo.tx ] TRACE: txid: 3 begin() thread: 198
+2015-02-11 18:24:05,343 [fluo.tx ] TRACE: txid: 3 class: com.SimpleLoader
+2015-02-11 18:24:05,357 [fluo.tx ] TRACE: txid: 3 get(4333, stat count ) -> null
+2015-02-11 18:24:05,357 [fluo.tx ] TRACE: txid: 3 set(4333, stat count , 1)
+2015-02-11 18:24:05,441 [fluo.tx ] TRACE: txid: 3 commit() -> SUCCESSFUL commitTs: 4
+2015-02-11 18:24:05,341 [fluo.tx ] TRACE: txid: 5 begin() thread: 198
+2015-02-11 18:24:05,442 [fluo.tx ] TRACE: txid: 3 close()
+2015-02-11 18:24:05,343 [fluo.tx ] TRACE: txid: 5 class: com.SimpleLoader
+2015-02-11 18:24:05,357 [fluo.tx ] TRACE: txid: 5 get(4333, stat count ) -> 1
+2015-02-11 18:24:05,357 [fluo.tx ] TRACE: txid: 5 set(4333, stat count , 2)
+2015-02-11 18:24:05,441 [fluo.tx ] TRACE: txid: 5 commit() -> SUCCESSFUL commitTs: 6
+2015-02-11 18:24:05,442 [fluo.tx ] TRACE: txid: 5 close()
+```
+
+The log above traces the following sequence of events.
+
+* Transaction T1 has a start timestamp of `3`
+* Thread with id `198` is executing T1, its running code from the class `com.SimpleLoader`
+* T1 reads row `4333` and column `stat count` which does not exist
+* T1 sets row `4333` and column `stat count` to `1`
+* T1 commits successfully and its commit timestamp from the Oracle is `4`.
+* Transaction T2 has a start timestamp of `5` (because its `5` > `4` it can see what T1 wrote).
+* T2 reads a value of `1` for row `4333` and column `stat count`
+* T2 sets row `4333` and `column `stat count` to `2`
+* T2 commits successfully with a commit timestamp of `6`
+
+Below is an example log after only setting `fluo.tx.collisions` to TRACE. This setting will only log
+trace information when a collision occurs. Unlike the previous example, what the transaction read
+and wrote is not logged. This shows that a transaction with a start timestamp of `106` and a class
+name of `com.SimpleLoader` collided with another transaction on row `r1` and column `fam1 qual1`.
+
+```
+2015-02-11 18:17:02,639 [tx.collisions] TRACE: txid: 106 class: com.SimpleLoader
+2015-02-11 18:17:02,639 [tx.collisions] TRACE: txid: 106 collisions: {r1=[fam1 qual1 ]}
+```
+
+When applications read and write arbitrary binary data, this does not log so well. In order to make
+the trace logs human readable, non ASCII chars are escaped using hex. The convention used it `\xDD`
+where D is a hex digit. Also the `\` character is escaped to make the output unambiguous.
+
+[FluoFactory]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/client/FluoFactory.html
+[FluoClient]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/client/FluoClient.html
+[FluoConfiguration]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/config/FluoConfiguration.html
+[Observer]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/observer/Observer.html
+[ObserverProvider]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/observer/ObserverProvider.html
+[fluo.properties]: https://github.com/apache/fluo/blob/rel/fluo-1.1.0-incubating/modules/distribution/src/main/config/fluo.properties
+[API]: https://fluo.apache.org/apidocs/
+[metrics]: /docs/fluo/1.1.0-incubating/metrics/
+[slf4j]: http://www.slf4j.org/
+[logback]: http://logback.qos.ch/
diff --git a/docs/fluo/1.1.0-incubating/architecture.md b/docs/fluo/1.1.0-incubating/architecture.md
new file mode 100644
index 0000000..b0755fb
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/architecture.md
@@ -0,0 +1,44 @@
+---
+layout: fluo-doc
+title:  Fluo Architecture
+version:  1.1.0-incubating
+---
+![fluo-architecture][1]
+
+## Fluo Application
+
+A **Fluo application** maintains a large scale computation using a series of small transactional
+updates. Fluo applications store their data in a **Fluo table** which has a similar structure (row,
+column, value) to an **Accumulo table** except that a Fluo table has no timestamps. A Fluo table
+is implemented using an Accumulo table. While you could scan the Accumulo table used to implement
+a Fluo table using an Accumulo client, you would read extra implementation-related data in addition
+to your data. Therefore, developers should only interact with the data in a Fluo table by writing
+Fluo client or observer code:
+
+* **Clients** ingest data or interact with Fluo from external applications (REST services,
+  crawlers, etc).
+* **Observers** are run by Fluo workers and trigger a transaction when a requested column is
+  modified in the Fluo table.
+
+Multiple Fluo applications can run on a cluster at the same time. Each Fluo application runs as a
+Hadoop YARN application and can be stopped, started, and upgraded independently. Fluo applications
+consist of an oracle process and a configurable number of worker processes:
+
+* The **Oracle** process allocates timestamps for transactions. While only one Oracle is required,
+  Fluo can be configured to run extra Oracles that can take over if the primary Oracle fails.
+* **Worker** processes run user code (called **observers**) that perform transactions. All workers
+   run the same observers. The number of worker instances are configured to handle the processing
+   workload.
+
+## Fluo Dependencies
+
+Fluo requires the following software to be running on the cluster:
+
+* **Accumulo** - Fluo stores its data in Accumulo and uses Accumulo's conditional mutations for
+  transactions.
+* **Hadoop** - Each Fluo application run its oracle and worker processes as Hadoop YARN
+  applications. HDFS is also required for Accumulo.
+* **Zookeeper** - Fluo stores its metadata and state information in Zookeeper. Zookeeper is also
+  required for Accumulo.
+
+[1]: /docs/fluo/1.1.0-incubating/resources/fluo-architecture.png
diff --git a/docs/fluo/1.1.0-incubating/contributing.md b/docs/fluo/1.1.0-incubating/contributing.md
new file mode 100644
index 0000000..ff4082a
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/contributing.md
@@ -0,0 +1,30 @@
+---
+layout: fluo-doc
+title:  Contributing to Fluo
+version:  1.1.0-incubating
+---
+## Building Fluo
+
+If you have [Git], [Maven], and [Java][java] (version 8+) installed, run these commands to build
+Fluo:
+
+    git clone https://github.com/apache/incubator-fluo.git
+    cd fluo
+    mvn package
+
+## Testing Fluo
+
+Fluo has a test suite that consists of the following:
+
+*  Units tests which are run by `mvn package`
+*  Integration tests which are run using `mvn verify`. These tests start a local Fluo instance
+   (called MiniFluo) and run against it.
+
+## See Also
+
+* [How to Contribute][contribute] on Apache Fluo project website
+
+[Git]: http://git-scm.com/
+[java]: http://openjdk.java.net/
+[Maven]: http://maven.apache.org/
+[contribute]: https://fluo.apache.org/how-to-contribute/
diff --git a/docs/fluo/1.1.0-incubating/grafana.md b/docs/fluo/1.1.0-incubating/grafana.md
new file mode 100644
index 0000000..c6dcf35
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/grafana.md
@@ -0,0 +1,83 @@
+---
+layout: fluo-doc
+title:  Fluo metrics in Grafana/InfluxDB
+version:  1.1.0-incubating
+---
+This document describes how to send Fluo metrics to [InfluxDB], a time series database, and make
+them viewable in [Grafana], a visualization tool. If you want general information on metrics, see
+the [Fluo metrics][2] documentation.
+
+## Set up Grafana/InfluxDB on your own
+
+Follow the instructions below to setup InfluxDB and Grafana.
+
+1.  Follow the standard installation instructions for [InfluxDB] and [Grafana]. As for versions,
+    the instructions below were written using InfluxDB v0.9.4.2 and Grafana v2.5.0.
+
+2.  Add the following to your InfluxDB configuration to configure it accept metrics in Graphite
+    format from Fluo. The configuration below contains templates that transform the Graphite
+    metrics into a format that is usable in InfluxDB.
+
+    ```
+    [[graphite]]
+      bind-address = ":2003"
+      enabled = true
+      database = "fluo_metrics"
+      protocol = "tcp"
+      consistency-level = "one"
+      separator = "_"
+      batch-size = 1000
+      batch-pending = 5
+      batch-timeout = "1s"
+      templates = [
+        "fluo.class.*.*.*.*.* ..app.host.measurement.observer.field",
+        "fluo.class.*.*.*.* ..app.host.measurement.observer",
+        "fluo.system.*.*.*.* ..app.host.measurement.field",
+        "fluo.system.*.*.* ..app.host.measurement",
+        "fluo.app.*.*.* ..host.measurement.field",
+        "fluo.app.*.* ..host.measurement",
+      ]
+    ```
+
+3. Fluo distributes a file called `fluo_metrics_setup.txt` that contains a list of commands that
+   setup InfluxDB. These commands will configure an InfluxDB user, retention policies, and
+   continuous queries that downsample data for the historical dashboard in Grafana. Run the command
+   below to execute the commands in this file:
+
+    ```
+    $INFLUXDB_HOME/bin/influx -import -path $FLUO_HOME/contrib/influxdb/fluo_metrics_setup.txt
+    ```
+
+3. Configure `fluo.properties` in your Fluo app configuration to send Graphite metrics to InfluxDB.
+   Below is example configuration. Remember to replace `<INFLUXDB_HOST>` with the actual host.
+
+    ```
+    fluo.metrics.reporter.graphite.enable=true
+    fluo.metrics.reporter.graphite.host=<INFLUXDB_HOST>
+    fluo.metrics.reporter.graphite.port=2003
+    fluo.metrics.reporter.graphite.frequency=30
+    ```
+
+    The reporting frequency of 30 sec is required if you are using the provided Grafana dashboards
+    that are configured in the next step.
+
+4.  Grafana needs to be configured to load dashboard JSON templates from a directory. Fluo
+    distributes two Grafana dashboard templates in its tarball distribution in the directory
+    `contrib/grafana`. Before restarting Grafana, you should copy the templates from your Fluo
+    installation to the `dashboards/` directory configured below.
+
+    ```
+    [dashboards.json]
+    enabled = true
+    path = <GRAFANA_HOME>/dashboards
+    ```
+
+5.  If you restart Grafana, you will see the Fluo dashboards configured but all of their charts will
+    be empty unless you have a Fluo application running and configured to send data to InfluxDB.
+    When you start sending data, you may need to refresh the dashboard page in the browser to start
+    viewing metrics.
+
+[1]: https://dropwizard.github.io/metrics/3.1.0/
+[2]: /docs/fluo/1.1.0-incubating/metrics/
+[Grafana]: http://grafana.org/
+[InfluxDB]: https://influxdb.com/
diff --git a/docs/fluo/1.1.0-incubating/index.md b/docs/fluo/1.1.0-incubating/index.md
new file mode 100644
index 0000000..9f5e520
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/index.md
@@ -0,0 +1,56 @@
+---
+layout: fluo-doc
+title:  Fluo 1.1.0-incubating Documentation
+version:  1.1.0-incubating
+---
+**Apache Fluo lets users make incremental updates to large data sets stored in Apache Accumulo.**
+
+[Apache Fluo][fluo] is an open source implementation of [Percolator][percolator] (which populates
+Google's search index) for [Apache Accumulo][accumulo]. Fluo makes it possible to update the results
+of a large-scale computation, index, or analytic as new data is discovered. Check out the Fluo
+[project website][fluo] for news and general information.
+
+## Getting Started
+
+* Take the [Fluo Tour][tour] if you are completely new to Fluo.
+* Read the [install instructions][install] to install Fluo and start a Fluo application in YARN on a
+  cluster where Accumulo, Hadoop & Zookeeper are running. If you need help setting up these
+  dependencies, see the [related projects page][related] for external projects that may help.
+
+## Applications
+
+Below are helpful resources for Fluo application developers:
+
+*  [Instructions][apps] for creating Fluo applications
+*  [Fluo API][api] javadocs
+*  [Fluo Recipes][recipes] is a project that provides common code for Fluo application developers
+   implemented using the Fluo API.
+
+## Implementation
+
+*  [Architecture] - Overview of Fluo's architecture
+*  [Contributing] - Documentation for developers who want to contribute to Fluo
+*  [Metrics] - Fluo metrics are visible via JMX by default but can be configured to send to Graphite
+   or Ganglia
+
+[fluo]: https://fluo.apache.org/
+[related]: https://fluo.apache.org/related-projects/
+[tour]: https://fluo.apache.org/tour/
+[accumulo]: https://accumulo.apache.org
+[percolator]: https://research.google.com/pubs/pub36726.html
+[install]: /docs/fluo/1.1.0-incubating/install/
+[apps]: /docs/fluo/1.1.0-incubating/applications/
+[api]: https://javadoc.io/doc/org.apache.fluo/fluo-api/1.1.0-incubating
+[recipes]: https://github.com/apache/incubator-fluo-recipes
+[Metrics]: /docs/fluo/1.1.0-incubating/metrics/
+[Contributing]: /docs/fluo/1.1.0-incubating/contributing/
+[Architecture]: /docs/fluo/1.1.0-incubating/architecture/
+[ti]: https://travis-ci.org/apache/incubator-fluo.svg?branch=master
+[tl]: https://travis-ci.org/apache/incubator-fluo
+[li]: http://img.shields.io/badge/license-ASL-blue.svg
+[ll]: https://github.com/apache/incubator-fluo/blob/master/LICENSE
+[mi]: https://maven-badges.herokuapp.com/maven-central/org.apache.fluo/fluo-api/badge.svg
+[ml]: https://maven-badges.herokuapp.com/maven-central/org.apache.fluo/fluo-api/
+[ji]: https://javadoc-emblem.rhcloud.com/doc/org.apache.fluo/fluo-api/badge.svg
+[jl]: http://www.javadoc.io/doc/org.apache.fluo/fluo-api
+[logo]: contrib/fluo-logo.png
diff --git a/docs/fluo/1.1.0-incubating/install.md b/docs/fluo/1.1.0-incubating/install.md
new file mode 100644
index 0000000..2b675a8
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/install.md
@@ -0,0 +1,238 @@
+---
+layout: fluo-doc
+title:  Fluo Install Instructions
+version:  1.1.0-incubating
+---
+Instructions for installing Apache Fluo and starting a Fluo application in YARN on a cluster where
+Accumulo, Hadoop & Zookeeper are running.  If you need help setting up these dependencies, see the
+[related projects page][related] for external projects that may help.
+
+## Requirements
+
+Before you install Fluo, the following software must be installed and running on your local machine
+or cluster:
+
+| Software    | Recommended Version | Minimum Version |
+|-------------|---------------------|-----------------|
+| [Accumulo]  | 1.7.2               | 1.6.1           |
+| [Hadoop]    | 2.7.2               | 2.6.0           |
+| [Zookeeper] | 3.4.8               |                 |
+| [Java]      | JDK 8               | JDK 8           |
+
+## Obtain a distribution
+
+Before you can install Fluo, you will need to obtain a distribution tarball. It is recommended that
+you download the [latest release][release]. You can also build a distribution from the master
+branch by following these steps which create a tarball in `modules/distribution/target`:
+
+    git clone https://github.com/apache/incubator-fluo.git
+    cd fluo/
+    mvn package
+
+## Install Fluo
+
+After you obtain a Fluo distribution tarball, follow these steps to install Fluo.
+
+1.  Choose a directory with plenty of space and untar the distribution:
+
+        tar -xvzf fluo-1.x.x-incubating-bin.tar.gz
+
+2.  Copy the example configuration to the base of your configuration directory to create the default
+    configuration for your Fluo install:
+
+        cp conf/examples/* conf/
+
+    The default configuration will be used as the base configuration for each new application.
+
+3.  Modify [fluo.properties] for your environment. However, you should not configure any
+    application settings (like observers).
+
+    NOTE - All properties that have a default are set with it. Uncomment a property if you want
+    to use a value different than the default. Properties that are unset and uncommented must be
+    set by the user.
+
+4. Fluo needs to build its classpath using jars from the versions of Hadoop, Accumulo, and
+Zookeeper that you are using. Choose one of the two ways below to make these jars available
+to Fluo:
+
+    * Set `HADOOP_PREFIX`, `ACCUMULO_HOME`, and `ZOOKEEPER_HOME` in your environment or configure
+    these variables in [fluo-env.sh]. Fluo will look in these locations for jars.
+    * Run `./lib/fetch.sh ahz` to download Hadoop, Accumulo, and Zookeeper jars to `lib/ahz` and
+    configure [fluo-env.sh] to look in this directory. By default, this command will download the
+    default versions set in [lib/ahz/pom.xml]. If you are not using the default versions, you can
+    override them:
+
+            ./lib/fetch.sh ahz -Daccumulo.version=1.7.2 -Dhadoop.version=2.7.2 -Dzookeeper.version=3.4.8
+
+5. Fluo needs more dependencies than what is available from Hadoop, Accumulo, and Zookeeper. These
+   extra dependencies need to be downloaded to `lib/` using the command below:
+
+        ./lib/fetch.sh extra
+
+You are now ready to use the Fluo command script.
+
+## Fluo command script
+
+The Fluo command script is located at `bin/fluo` of your Fluo installation. All Fluo commands are
+invoked by this script.
+
+Modify and add the following to your `~/.bashrc` if you want to be able to execute the fluo script
+from any directory:
+
+    export PATH=/path/to/fluo-1.x.x-incubating/bin:$PATH
+
+Source your `.bashrc` for the changes to take effect and test the script
+
+    source ~/.bashrc
+    fluo
+
+Running the script without any arguments prints a description of all commands.
+
+    ./bin/fluo
+
+## Configure a Fluo application
+
+You are now ready to configure a Fluo application. Use the command below to create the
+configuration necessary for a new application. Feel free to pick a different name (other than
+`myapp`) for your application:
+
+    fluo new myapp
+
+This command will create a directory for your application at `apps/myapp` of your Fluo install which
+will contain a `conf` and `lib`.
+
+The `apps/myapp/conf` directory contains a copy of the `fluo.properties` from your default
+configuration. This should be configured for your application:
+
+    vim apps/myapp/fluo.properties
+
+When configuring the observer section in fluo.properties, you can configure your instance for the
+[phrasecount] application if you have not created your own application. See the [phrasecount]
+example for instructions. You can also choose not to configure any observers but your workers will
+be idle when started.
+
+The `apps/myapp/lib` directory should contain any observer jars for your application. If you
+configured [fluo.properties] for observers, copy any jars containing these observer classes to this
+directory.
+
+## Initialize your application
+
+After your application has been configured, use the command below to initialize it:
+
+    fluo init myapp
+
+This only needs to be called once and stores configuration in Zookeeper.
+
+## Start your application
+
+A Fluo application consists of one oracle process and multiple worker processes. Before starting
+your application, you can configure the number of worker process in your [fluo.properties] file.
+
+When you are ready to start your Fluo application on your YARN cluster, run the command below:
+
+    fluo start myapp
+
+The start command above will work for a single-node or a large cluster. By using YARN, you do not
+need to deploy the Fluo binaries to every node on your cluster or start processes on every node.
+
+You can use the following command to check the status of your instance:
+
+    fluo status myapp
+
+For more detailed information on the YARN containers running Fluo:
+
+    fluo info myapp
+
+You can also use `yarn application -list` to check the status of your Fluo instance in YARN. 
+
+## View Fluo application logs
+
+Fluo application logs are viewable within YARN using the methods below:
+
+1. View logs using the web interface of the the YARN resource manager
+(`http://<resource manager>:8088/cluster`). First, click on the application ID (i.e `application_*`)
+of your Fluo application and then click on the latest attempt ID (i.e `appattempt_*`). You should
+see a list of containers. There should be a container for the application master (typically
+container 1), a Fluo oracle (typically container 2), and Fluo workers (containers 3+). You can view
+the log files produced by a container by clicking on its 'logs' link. Logs from Fluo observers will
+be in the `worker_*.log` file for each of your worker containers. 
+
+2. View logs on disk in the directory specified by the YARN property `yarn.nodemanager.log-dirs` in
+your YARN configuration `yarn-site.xml` (see [yarn-default.xml] for more info about this property).
+If you are running Fluo on single machine, this method works well. If you are running Fluo on a
+cluster, your containers and their logs will be spread across the cluster.
+
+When running Fluo in YARN, Fluo must use [logback] for logging (due to a hard requirement by Twill).
+Logback is configured using `/path/to/fluo/apps/<app_name>/conf/logback.xml`. You should review this
+configuration but the root logger is configured by default to print any message that is debug level
+or higher.
+
+In addition the `*.log` files, Fluo oracle and worker containers will have `stdout` and `stderr`
+files. These files may have helpful error messages. Especially, if a process failed to start
+or calls were made to `System.out` or `System.err` in your application.
+
+## View Fluo application data
+
+When you have data in your Fluo application, you can view it using the command `fluo scan`. Pipe the
+output to `less` using the command `fluo scan | less` if you want to page through the data.
+
+## Stop your Fluo application
+
+Use the following command to stop your Fluo application:
+
+    fluo stop myapp
+
+If stop fails, there is also a kill command.
+
+    fluo kill myapp
+
+## Tuning Accumulo
+
+Fluo will reread the same data frequently when it checks conditions on mutations. When Fluo
+initializes a table it enables data caching to make this more efficient. However you may need to
+increase the amount of memory available for caching in the tserver by increasing
+`tserver.cache.data.size`. Increasing this may require increasing the maximum tserver java heap size
+in `accumulo-env.sh`.
+
+Fluo will run many client threads, will want to ensure the tablet server has enough threads. Should
+probably increase the `tserver.server.threads.minimum` Accumulo setting.
+
+Using at least Accumulo 1.6.1 is recommended because multiple performance bugs were fixed.
+
+## Tuning YARN
+
+When running Fluo oracles and workers in YARN, the number of instances, max memory, and number of
+cores for Fluo processes can be configured in [fluo.properties]. If YARN is killing processes
+consider increasing `twill.java.reserved.memory.mb` (which defaults to 200 and is set in
+yarn-site.xml). The `twill.java.reserved.memory.mb` config determines the gap between the YARN
+memory limit set in [fluo.properties] and the java -Xmx setting. For example, if max memory is 1024
+and twill reserved memory is 200, the java -Xmx setting will be 1024-200 = 824 MB.
+
+## Run locally without YARN
+
+If you do not have YARN set up, you can start the oracle and worker as a local Fluo process using
+the following commands:
+
+    local-fluo start-oracle
+    local-fluo start-worker
+
+Use the following commands to stop a local Fluo process:
+
+    local-fluo stop-worker
+    local-fluo stop-oracle
+
+In a distributed environment, you will need to deploy and configure a Fluo distribution on every
+node in your cluster.
+
+[Accumulo]: https://accumulo.apache.org/
+[Hadoop]: http://hadoop.apache.org/
+[Zookeeper]: http://zookeeper.apache.org/
+[Java]: http://openjdk.java.net/
+[related]: https://fluo.apache.org/related-projects/
+[release]: https://fluo.apache.org/download/
+[phrasecount]: https://github.com/fluo-io/phrasecount
+[fluo.properties]: https://github.com/apache/fluo/blob/rel/fluo-1.1.0-incubating/modules/distribution/src/main/config/fluo.properties
+[fluo-env.sh]: https://github.com/apache/fluo/blob/rel/fluo-1.1.0-incubating/modules/distribution/src/main/config/fluo-env.sh
+[lib/ahz/pom.xml]: https://github.com/apache/fluo/blob/rel/fluo-1.1.0-incubating/modules/distribution/src/main/lib/ahz/pom.xml
+[yarn-default.xml]: https://hadoop.apache.org/docs/r2.7.0/hadoop-yarn/hadoop-yarn-common/yarn-default.xml
+[logback]: http://logback.qos.ch/
diff --git a/docs/fluo/1.1.0-incubating/metrics.md b/docs/fluo/1.1.0-incubating/metrics.md
new file mode 100644
index 0000000..c87b670
--- /dev/null
+++ b/docs/fluo/1.1.0-incubating/metrics.md
@@ -0,0 +1,116 @@
+---
+layout: fluo-doc
+title:  Fluo Metrics
+version:  1.1.0-incubating
+---
+A Fluo application can be configured (in [fluo.properties]) to report metrics. When metrics are
+configured, Fluo will report some 'default' metrics about an application that help users monitor its
+performance. Users can also write code to report 'application-specific' metrics from their
+applications. Both 'application-specific' and 'default' metrics share the same reporter configured
+by [fluo.properties] and are described in detail below.
+
+## Configuring reporters
+
+Fluo metrics are not published by default. To publish metrics, configure a reporter in the 'metrics'
+section of [fluo.properties]. There are several different reporter types (i.e Console, CSV,
+Graphite, JMX, SLF4J) that are implemented using [Dropwizard]. The choice of which reporter to use
+depends on the visualization tool used. If you are not currently using a visualization tool, there
+is [documentation][grafana] for reporting Fluo metrics to Grafana/InfluxDB.
+
+## Metrics names
+
+When Fluo metrics are reported, they are published using a naming scheme that encodes additional
+information. This additional information is represented using all caps variables (i.e `METRIC`)
+below.
+
+Default metrics start with `fluo.class` or `fluo.system` and have following naming schemes:
+
+        fluo.class.APPLICATION.REPORTER_ID.METRIC.CLASS
+        fluo.system.APPLICATION.REPORTER_ID.METRIC
+
+Application metrics start with `fluo.app` and have following scheme:
+
+        fluo.app.REPORTER_ID.METRIC
+
+The variables below describe the additional information that is encoded in metrics names.
+
+1. `APPLICATION` - Fluo application name
+2. `REPORTER_ID` - Unique ID of the Fluo oracle, worker, or client that is reporting the metric.
+    When running in YARN, this ID is of the format `worker-INSTANCE_ID` or `oracle-INSTANCE_ID`
+    where `INSTANCE_ID` corresponds to instance number. When not running in YARN, this ID consists
+    of a hostname and a base36 long that is unique across all fluo processes.
+3. `METRIC` - Name of the metric. For 'default' metrics, this is set by Fluo. For 'application'
+    metrics, this is set by user. Name should be unique and avoid using period '.' in name.
+4. `CLASS` - Name of Fluo observer or loader class that produced metric. This allows things like
+    transaction collisions to be tracked per class.
+
+## Application-specific metrics
+
+Application metrics are implemented by retrieving a [MetricsReporter] from an [Observer], [Loader],
+or [FluoClient].  These metrics are named using the format `fluo.app.REPORTER_ID.METRIC`.
+
+## Default metrics
+
+Default metrics report for a particular Observer/Loader class or system-wide.
+
+Below are metrics that are reported from each Observer/Loader class that is configured in a Fluo
+application. These metrics are reported after each transaction and named using the format
+`fluo.class.APPLICATION.REPORTER_ID.METRIC.CLASS`.
+
+* tx_lock_wait_time - [Timer]
+    - Time transaction spent waiting on locks held by other transactions.
+    - Only updated for transactions that have non-zero lock time.
+* tx_execution_time - [Timer]
+    - Time transaction took to execute.
+    - Updated for failed and successful transactions.
+    - This does not include commit time, only the time from start until commit is called.
+* tx_with_collision - [Meter]
+    - Rate of transactions with collisions.
+* tx_collisions - [Meter]
+    - Rate of collisions.
+* tx_entries_set - [Meter]
+    - Rate of row/columns set by transaction
+* tx_entries_read - [Meter]
+    - Rate of row/columns read by transaction that existed.
+    - There is currently no count of all reads (including non-existent data)
+* tx_locks_timedout - [Meter]
+    - Rate of timedout locks rolled back by transaction.
+    - These are locks that are held for very long periods by another transaction that appears to be
+      alive based on zookeeper.
+* tx_locks_dead - [Meter]
+    - Rate of dead locks rolled by a transaction.
+    - These are locks held by a process that appears to be dead according to zookeeper.
+* tx_status_`<STATUS>` - [Meter]
+    - Rate of different ways (i.e `<STATUS>`) a transaction can terminate
+
+Below are system-wide metrics that are reported for the entire Fluo application. These metrics are
+named using the format `fluo.system.APPLICATION.REPORTER_ID.METRIC`.
+
+* oracle_response_time - [Timer]
+    - Time each RPC call to oracle for stamps took
+* oracle_client_stamps - [Histogram]
+    - Number of stamps requested for each request for stamps to the server
+* oracle_server_stamps - [Histogram]
+    - Number of stamps requested for each request for stamps from a client
+* worker_notifications_queued - [Gauge]
+    - The current number of notifications queued for processing.
+* transactor_committing - [Gauge]
+    - The current number of transactions that are working their way through the commit steps.
+
+Histograms and Timers have a counter. In the case of a histogram, the counter is the number of times
+the metric was updated and not a sum of the updates. For example if a request for 5 timestamps was
+made to the oracle followed by a request for 3 timestamps, then the count for `oracle_server_stamps`
+would be 2 and the mean would be (5+3)/2.
+
+[fluo.properties]: https://github.com/apache/fluo/blob/rel/fluo-1.1.0-incubating/modules/distribution/src/main/config/fluo.properties
+[Dropwizard]: https://dropwizard.github.io/metrics/3.1.0/
+[grafana]: /docs/fluo/1.1.0-incubating/grafana/
+[MetricsReporter]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/metrics/MetricsReporter.html
+[Observer]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/observer/Observer.html
+[Loader]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/client/Loader.html
+[FluoClient]: {{ site.fluo_api_static }}/1.1.0-incubating/org/apache/fluo/api/client/FluoClient.html
+[Timer]: https://dropwizard.github.io/metrics/3.1.0/getting-started/#timers
+[Counter]: https://dropwizard.github.io/metrics/3.1.0/getting-started/#counters
+[Histogram]: https://dropwizard.github.io/metrics/3.1.0/getting-started/#histograms
+[Gauge]: https://dropwizard.github.io/metrics/3.1.0/getting-started/#gauges
+[Meter]: https://dropwizard.github.io/metrics/3.1.0/getting-started/#meters
diff --git a/docs/fluo/1.1.0-incubating/resources/fluo-architecture.png b/docs/fluo/1.1.0-incubating/resources/fluo-architecture.png
new file mode 100644
index 0000000..3ba96fd
Binary files /dev/null and b/docs/fluo/1.1.0-incubating/resources/fluo-architecture.png differ
diff --git a/docs/index.md b/docs/index.md
index d519c0d..3fffa73 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -12,6 +12,7 @@ For a general overview of Fluo, take the [Fluo tour](/tour/).
 
 ### Apache Fluo documentation
 
+* [1.1.0-incubating][fluo-1.1] - June 12, 2017
 * [1.0.0-incubating][fluo-1.0] - October 14, 2016
 
 ### Apache Fluo Recipes documentation
@@ -22,5 +23,6 @@ Documentation for releases before joining Apache have been [archived](archive).
 
 [Apache Fluo]: https://github.com/apache/fluo
 [Apache Fluo Recipes]: https://github.com/apache/fluo-recipes
+[fluo-1.1]: /docs/fluo/1.1.0-incubating/
 [fluo-1.0]: /docs/fluo/1.0.0-incubating/
 [recipes-1.0]: /docs/fluo-recipes/1.0.0-incubating/

-- 
To stop receiving notification emails like this one, please contact
['"commits@fluo.apache.org" <co...@fluo.apache.org>'].