You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gearpump.apache.org by ma...@apache.org on 2016/09/29 12:06:27 UTC

[06/11] incubator-gearpump git commit: [GEARPUMP-213] build docs with MkDocs

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/dev/dev-write-1st-app.md
----------------------------------------------------------------------
diff --git a/docs/docs/dev/dev-write-1st-app.md b/docs/docs/dev/dev-write-1st-app.md
new file mode 100644
index 0000000..f28a303
--- /dev/null
+++ b/docs/docs/dev/dev-write-1st-app.md
@@ -0,0 +1,370 @@
+We'll use [wordcount](https://github.com/apache/incubator-gearpump/tree/master/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount) as an example to illustrate how to write Gearpump applications.
+
+### Maven/Sbt Settings
+
+Repository and library dependencies can be found at [Maven Setting](http://gearpump.apache.org/downloads.html#maven-dependencies).
+
+### IDE Setup (Optional)
+You can get your preferred IDE ready for Gearpump by following [this guide](dev-ide-setup).
+
+### Decide which language and API to use for writing 
+Gearpump supports two level APIs:
+
+1. Low level API, which is more similar to Akka programming, operating on each event. The API document can be found at [Low Level API Doc](http://gearpump.apache.org/releases/latest/api/scala/index.html#org.apache.gearpump.streaming.package).
+
+2. High level API (aka DSL), which is operating on streaming instead of individual event. The API document can be found at [DSL API Doc](http://gearpump.apache.org/releases/latest/api/scala/index.html#org.apache.gearpump.streaming.dsl.package).
+
+And both APIs have their Java version and Scala version.
+
+So, before you writing your first Gearpump application, you need to decide which API to use and which language to use. 
+
+## DSL version for Wordcount
+
+The easiest way to write your streaming application is to write it with Gearpump DSL. 
+Below will demostrate how to write WordCount application via Gearpump DSL.
+
+#### In Scala
+
+	:::scala     
+	/** WordCount with High level DSL */
+	object WordCount extends AkkaApp with ArgumentsParser {
+	
+	  override val options: Array[(String, CLIOption[Any])] = Array.empty
+	
+	  override def main(akkaConf: Config, args: Array[String]): Unit = {
+	    val context = ClientContext(akkaConf)
+	    val app = StreamApp("dsl", context)
+	    val data = "This is a good start, bingo!! bingo!!"
+	
+	    //count for each word and output to log
+	    app.source(data.lines.toList, 1, "source").
+	      // word => (word, count)
+	      flatMap(line => line.split("[\\s]+")).map((_, 1)).
+	      // (word, count1), (word, count2) => (word, count1 + count2)
+	      groupByKey().sum.log
+	
+	    val appId = context.submit(app)
+	    context.close()
+	  }
+	}
+	
+
+#### In Java
+
+	:::java   
+	/** Java version of WordCount with high level DSL API */
+	public class WordCount {
+	
+	  public static void main(String[] args) throws InterruptedException {
+	    main(ClusterConfig.defaultConfig(), args);
+	  }
+	
+	  public static void main(Config akkaConf, String[] args) throws InterruptedException {
+	    ClientContext context = new ClientContext(akkaConf);
+	    JavaStreamApp app = new JavaStreamApp("JavaDSL", context, UserConfig.empty());
+	    List<String> source = Lists.newArrayList("This is a good start, bingo!! bingo!!");
+	
+	    //create a stream from the string list.
+	    JavaStream<String> sentence = app.source(source, 1, UserConfig.empty(), "source");
+	
+	    //tokenize the strings and create a new stream
+	    JavaStream<String> words = sentence.flatMap(new FlatMapFunction<String, String>() {
+	      @Override
+	      public Iterator<String> apply(String s) {
+	        return Lists.newArrayList(s.split("\\s+")).iterator();
+	      }
+	    }, "flatMap");
+	
+	    //map each string as (string, 1) pair
+	    JavaStream<Tuple2<String, Integer>> ones = words.map(new MapFunction<String, Tuple2<String, Integer>>() {
+	      @Override
+	      public Tuple2<String, Integer> apply(String s) {
+	        return new Tuple2<String, Integer>(s, 1);
+	      }
+	    }, "map");
+	
+	    //group by according to string
+	    JavaStream<Tuple2<String, Integer>> groupedOnes = ones.groupBy(new GroupByFunction<Tuple2<String, Integer>, String>() {
+	      @Override
+	      public String apply(Tuple2<String, Integer> tuple) {
+	        return tuple._1();
+	      }
+	    }, 1, "groupBy");
+	
+	    //for each group, make the sum
+	    JavaStream<Tuple2<String, Integer>> wordcount = groupedOnes.reduce(new ReduceFunction<Tuple2<String, Integer>>() {
+	      @Override
+	      public Tuple2<String, Integer> apply(Tuple2<String, Integer> t1, Tuple2<String, Integer> t2) {
+	        return new Tuple2<String, Integer>(t1._1(), t1._2() + t2._2());
+	      }
+	    }, "reduce");
+	
+	    //output result using log
+	    wordcount.log();
+	
+	    app.run();
+	    context.close();
+	  }
+	}
+
+
+## Low level API based Wordcount
+
+### Define Processor(Task) class and Partitioner class
+
+An application is a Directed Acyclic Graph (DAG) of processors. In the wordcount example, We will firstly define two processors `Split` and `Sum`, and then weave them together.
+
+
+#### Split processor
+
+In the `Split` processor, we simply split a predefined text (the content is simplified for conciseness) and send out each split word to `Sum`.
+
+#### In Scala
+
+	:::scala
+	class Split(taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
+	  import taskContext.output
+	
+	  override def onStart(startTime : StartTime) : Unit = {
+	    self ! Message("start")
+	  }
+	
+	  override def onNext(msg : Message) : Unit = {
+	    Split.TEXT_TO_SPLIT.lines.foreach { line =>
+	      line.split("[\\s]+").filter(_.nonEmpty).foreach { msg =>
+	        output(new Message(msg, System.currentTimeMillis()))
+	      }
+	    }
+	    self ! Message("continue", System.currentTimeMillis())
+	  }
+	}
+	
+	object Split {
+	  val TEXT_TO_SPLIT = "some text"
+	}
+
+#### In Java
+
+	:::java
+	public class Split extends Task {
+	
+	  public static String TEXT = "This is a good start for java! bingo! bingo! ";
+	
+	  public Split(TaskContext taskContext, UserConfig userConf) {
+	    super(taskContext, userConf);
+	  }
+	
+	  private Long now() {
+	    return System.currentTimeMillis();
+	  }
+	
+	  @Override
+	  public void onStart(StartTime startTime) {
+	    self().tell(new Message("start", now()), self());
+	  }
+	
+	  @Override
+	  public void onNext(Message msg) {
+	
+	    // Split the TEXT to words
+	    String[] words = TEXT.split(" ");
+	    for (int i = 0; i < words.length; i++) {
+	      context.output(new Message(words[i], now()));
+	    }
+	    self().tell(new Message("next", now()), self());
+	  }
+	}
+	```
+
+
+Essentially, each processor consists of two descriptions:
+
+1. A `Task` to define the operation.
+
+2. A parallelism level to define the number of tasks of this processor in parallel. 
+ 
+Just like `Split`, every processor extends `Task`.  The `onStart` method is called once before any message comes in; `onNext` method is called to process every incoming message. Note that Gearpump employs the message-driven model and that's why Split sends itself a message at the end of `onStart` and `onNext` to trigger next message processing.
+
+#### Sum Processor
+
+The structure of `Sum` processor looks much alike. `Sum` does not need to send messages to itself since it receives messages from `Split`.
+
+#### In Scala
+
+	:::scala
+	class Sum (taskContext : TaskContext, conf: UserConfig) extends Task(taskContext, conf) {
+	  private[wordcount] val map : mutable.HashMap[String, Long] = new mutable.HashMap[String, Long]()
+	
+	  private[wordcount] var wordCount : Long = 0
+	  private var snapShotTime : Long = System.currentTimeMillis()
+	  private var snapShotWordCount : Long = 0
+	
+	  private var scheduler : Cancellable = null
+	
+	  override def onStart(startTime : StartTime) : Unit = {
+	    scheduler = taskContext.schedule(new FiniteDuration(5, TimeUnit.SECONDS),
+	      new FiniteDuration(5, TimeUnit.SECONDS))(reportWordCount)
+	  }
+	
+	  override def onNext(msg : Message) : Unit = {
+	    if (null == msg) {
+	      return
+	    }
+	    val current = map.getOrElse(msg.msg.asInstanceOf[String], 0L)
+	    wordCount += 1
+	    map.put(msg.msg.asInstanceOf[String], current + 1)
+	  }
+	
+	  override def onStop() : Unit = {
+	    if (scheduler != null) {
+	      scheduler.cancel()
+	    }
+	  }
+	
+	  def reportWordCount() : Unit = {
+	    val current : Long = System.currentTimeMillis()
+	    LOG.info(s"Task ${taskContext.taskId} Throughput: ${(wordCount - snapShotWordCount, (current - snapShotTime) / 1000)} (words, second)")
+	    snapShotWordCount = wordCount
+	    snapShotTime = current
+	  }
+	}
+	
+#### In Java
+
+	:::java
+	public class Sum extends Task {
+	
+	  private Logger LOG = super.LOG();
+	  private HashMap<String, Integer> wordCount = new HashMap<String, Integer>();
+	
+	  public Sum(TaskContext taskContext, UserConfig userConf) {
+	    super(taskContext, userConf);
+	  }
+	
+	  @Override
+	  public void onStart(StartTime startTime) {
+	    //skip
+	  }
+	
+	  @Override
+	  public void onNext(Message messagePayLoad) {
+	    String word = (String) (messagePayLoad.msg());
+	    Integer current = wordCount.get(word);
+	    if (current == null) {
+	      current = 0;
+	    }
+	    Integer newCount = current + 1;
+	    wordCount.put(word, newCount);
+	  }
+	}
+
+
+Besides counting the sum, in Scala version, we also define a scheduler to report throughput every 5 seconds. The scheduler should be cancelled when the computation completes, which could be accomplished overriding the `onStop` method. The default implementation of `onStop` is a no-op.
+
+#### Partitioner
+
+A processor could be parallelized to a list of tasks. A `Partitioner` defines how the data is shuffled among tasks of Split and Sum. Gearpump has already provided two partitioners
+
+* `HashPartitioner`: partitions data based on the message's hashcode
+* `ShufflePartitioner`: partitions data in a round-robin way.
+
+You could define your own partitioner by extending the `Partitioner` trait/interface and overriding the `getPartition` method.
+
+
+	:::scala
+	trait Partitioner extends Serializable {
+	  def getPartition(msg : Message, partitionNum : Int) : Int
+	}
+
+### Wrap up as an application 
+
+Now, we are able to write our application class, weaving the above components together.
+
+The application class extends `App` and `ArgumentsParser which make it easier to parse arguments and run main functions.
+
+#### In Scala
+
+	:::scala
+	object WordCount extends App with ArgumentsParser {
+	  private val LOG: Logger = LogUtil.getLogger(getClass)
+	  val RUN_FOR_EVER = -1
+	
+	  override val options: Array[(String, CLIOption[Any])] = Array(
+	    "split" -> CLIOption[Int]("<how many split tasks>", required = false, defaultValue = Some(1)),
+	    "sum" -> CLIOption[Int]("<how many sum tasks>", required = false, defaultValue = Some(1))
+	  )
+	
+	  def application(config: ParseResult) : StreamApplication = {
+	    val splitNum = config.getInt("split")
+	    val sumNum = config.getInt("sum")
+	    val partitioner = new HashPartitioner()
+	    val split = Processor[Split](splitNum)
+	    val sum = Processor[Sum](sumNum)
+	    val app = StreamApplication("wordCount", Graph[Processor[_ <: Task], Partitioner](split ~ partitioner ~> sum), UserConfig.empty)
+	    app
+	  }
+	
+	  val config = parse(args)
+	  val context = ClientContext()
+	  val appId = context.submit(application(config))
+	  context.close()
+	}
+	
+
+
+We override `options` value and define an array of command line arguments to parse. We want application users to pass in masters' hosts and ports, the parallelism of split and sum tasks, and how long to run the example. We also specify whether an option is `required` and provide `defaultValue` for some arguments.
+
+#### In Java
+
+	:::java
+	
+	/** Java version of WordCount with Processor Graph API */
+	public class WordCount {
+	
+	  public static void main(String[] args) throws InterruptedException {
+	    main(ClusterConfig.defaultConfig(), args);
+	  }
+	
+	  public static void main(Config akkaConf, String[] args) throws InterruptedException {
+	
+	    // For split task, we config to create two tasks
+	    int splitTaskNumber = 2;
+	    Processor split = new Processor(Split.class).withParallelism(splitTaskNumber);
+	
+	    // For sum task, we have two summer.
+	    int sumTaskNumber = 2;
+	    Processor sum = new Processor(Sum.class).withParallelism(sumTaskNumber);
+	
+	    // construct the graph
+	    Graph graph = new Graph();
+	    graph.addVertex(split);
+	    graph.addVertex(sum);
+	
+	    Partitioner partitioner = new HashPartitioner();
+	    graph.addEdge(split, partitioner, sum);
+	
+	    UserConfig conf = UserConfig.empty();
+	    StreamApplication app = new StreamApplication("wordcountJava", conf, graph);
+	
+	    // create master client
+	    // It will read the master settings under gearpump.cluster.masters
+	    ClientContext masterClient = new ClientContext(akkaConf);
+	
+	    masterClient.submit(app);
+	
+	    masterClient.close();
+	  }
+	}
+
+
+## Submit application
+
+After all these, you need to package everything into a uber jar and submit the jar to Gearpump Cluster. Please check [Application submission tool](../introduction/commandline) to command line tool syntax.
+
+## Advanced topic
+For a real application, you definitely need to define your own customized message passing between processors.
+Customized message needs customized serializer to help message passing over wire.
+Check [this guide](dev-custom-serializer) for how to customize serializer.
+
+### Gearpump for Non-Streaming Usage
+Gearpump is also able to as a base platform to develop non-streaming applications. See [this guide](dev-non-streaming-example) on how to use Gearpump to develop a distributed shell.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/actor_hierarchy.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/actor_hierarchy.png b/docs/docs/img/actor_hierarchy.png
new file mode 100644
index 0000000..d971745
Binary files /dev/null and b/docs/docs/img/actor_hierarchy.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/checkpoint_equation.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/checkpoint_equation.png b/docs/docs/img/checkpoint_equation.png
new file mode 100644
index 0000000..14da93b
Binary files /dev/null and b/docs/docs/img/checkpoint_equation.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/checkpoint_interval_equation.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/checkpoint_interval_equation.png b/docs/docs/img/checkpoint_interval_equation.png
new file mode 100644
index 0000000..0c0414c
Binary files /dev/null and b/docs/docs/img/checkpoint_interval_equation.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/checkpointing.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/checkpointing.png b/docs/docs/img/checkpointing.png
new file mode 100644
index 0000000..f11eb53
Binary files /dev/null and b/docs/docs/img/checkpointing.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/checkpointing_interval.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/checkpointing_interval.png b/docs/docs/img/checkpointing_interval.png
new file mode 100644
index 0000000..dc46317
Binary files /dev/null and b/docs/docs/img/checkpointing_interval.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/clock.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/clock.png b/docs/docs/img/clock.png
new file mode 100644
index 0000000..906d51d
Binary files /dev/null and b/docs/docs/img/clock.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/dag.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/dag.png b/docs/docs/img/dag.png
new file mode 100644
index 0000000..c0ca79f
Binary files /dev/null and b/docs/docs/img/dag.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/dashboard.gif
----------------------------------------------------------------------
diff --git a/docs/docs/img/dashboard.gif b/docs/docs/img/dashboard.gif
new file mode 100644
index 0000000..0170c5f
Binary files /dev/null and b/docs/docs/img/dashboard.gif differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/dashboard.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/dashboard.png b/docs/docs/img/dashboard.png
new file mode 100644
index 0000000..0b5eedd
Binary files /dev/null and b/docs/docs/img/dashboard.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/dashboard_3.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/dashboard_3.png b/docs/docs/img/dashboard_3.png
new file mode 100644
index 0000000..47259fc
Binary files /dev/null and b/docs/docs/img/dashboard_3.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/download.jpg
----------------------------------------------------------------------
diff --git a/docs/docs/img/download.jpg b/docs/docs/img/download.jpg
new file mode 100644
index 0000000..7129c52
Binary files /dev/null and b/docs/docs/img/download.jpg differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/dynamic.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/dynamic.png b/docs/docs/img/dynamic.png
new file mode 100644
index 0000000..09b8a35
Binary files /dev/null and b/docs/docs/img/dynamic.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/exact.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/exact.png b/docs/docs/img/exact.png
new file mode 100644
index 0000000..f11eb53
Binary files /dev/null and b/docs/docs/img/exact.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/failures.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/failures.png b/docs/docs/img/failures.png
new file mode 100644
index 0000000..fa98cdc
Binary files /dev/null and b/docs/docs/img/failures.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/flow_control.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/flow_control.png b/docs/docs/img/flow_control.png
new file mode 100644
index 0000000..7ea9bd2
Binary files /dev/null and b/docs/docs/img/flow_control.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/flowcontrol.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/flowcontrol.png b/docs/docs/img/flowcontrol.png
new file mode 100644
index 0000000..7ea9bd2
Binary files /dev/null and b/docs/docs/img/flowcontrol.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/ha.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/ha.png b/docs/docs/img/ha.png
new file mode 100644
index 0000000..5474d84
Binary files /dev/null and b/docs/docs/img/ha.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/kafka_wordcount.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/kafka_wordcount.png b/docs/docs/img/kafka_wordcount.png
new file mode 100644
index 0000000..a43fa55
Binary files /dev/null and b/docs/docs/img/kafka_wordcount.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/layout.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/layout.png b/docs/docs/img/layout.png
new file mode 100644
index 0000000..edffdf8
Binary files /dev/null and b/docs/docs/img/layout.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/logo.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/logo.png b/docs/docs/img/logo.png
new file mode 100644
index 0000000..7575892
Binary files /dev/null and b/docs/docs/img/logo.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/logo.svg
----------------------------------------------------------------------
diff --git a/docs/docs/img/logo.svg b/docs/docs/img/logo.svg
new file mode 100644
index 0000000..5897ca4
--- /dev/null
+++ b/docs/docs/img/logo.svg
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   width="89mm"
+   height="89mm"
+   viewBox="0 0 89 89"
+   id="svg4684"
+   version="1.1"
+   inkscape:version="0.91 r13725"
+   sodipodi:docname="gearpump_logo_simple.svg"
+   inkscape:export-filename="/Users/kamkasravi/Dropbox/gearpump/gearpump_logo_simple.png"
+   inkscape:export-xdpi="157.11"
+   inkscape:export-ydpi="157.11">
+  <defs
+     id="defs4686" />
+  <sodipodi:namedview
+     inkscape:document-units="mm"
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="0"
+     inkscape:pageopacity="0"
+     inkscape:pageshadow="2"
+     inkscape:zoom="1.2162434"
+     inkscape:cx="395.86179"
+     inkscape:cy="372.04726"
+     inkscape:current-layer="layer1"
+     id="namedview4688"
+     showgrid="false"
+     inkscape:window-width="1920"
+     inkscape:window-height="1107"
+     inkscape:window-x="0"
+     inkscape:window-y="1"
+     inkscape:window-maximized="1"
+     fit-margin-top="0"
+     fit-margin-left="0"
+     fit-margin-right="0"
+     fit-margin-bottom="0"
+     inkscape:showpageshadow="false"
+     showborder="true" />
+  <metadata
+     id="metadata4690">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <g
+     inkscape:label="Layer 1"
+     inkscape:groupmode="layer"
+     id="layer1"
+     transform="translate(5.4566873e-7,-121)">
+    <path
+       style="fill:#556ae6;fill-opacity:1;stroke:none;stroke-width:7.08661413;stroke-miterlimit:4;stroke-dasharray:none"
+       d="m 59.451523,122.95775 a 44.775326,44.775326 0 0 0 -21.03861,-2.01136 l -0.6557,4.60251 a 23.524902,23.524902 0 0 1 4.20498,3.77358 l 6.73498,-1.78044 a 29.907628,29.907628 0 0 1 4.33792,8.99185 l -5.54364,4.16221 a 23.524902,23.524902 0 0 1 0.32108,5.63603 l 6.03185,3.50988 a 29.907628,29.907628 0 0 1 -1.23578,4.84982 29.907628,29.907628 0 0 1 -2.06484,4.5672 l -6.85567,-0.97589 a 23.524902,23.524902 0 0 1 -3.77339,4.20446 l 1.78078,6.73569 a 29.907628,29.907628 0 0 1 -8.99151,4.33863 l -4.16256,-5.54435 a 23.524902,23.524902 0 0 1 -5.63602,0.32108 l -3.50989,6.03185 a 29.907628,29.907628 0 0 1 -4.849808,-1.23578 29.907628,29.907628 0 0 1 -4.5675477,-2.06554 l 0.9757077,-6.85515 a 23.524902,23.524902 0 0 1 -4.2041157,-3.7727 l -6.64489998,1.75628 a 44.775326,44.775326 0 0 0 29.97375368,45.3569 44.775326,44.775326 0 0 0 27.28111,0.61075 l -0.0231,-0.19065 a 23.524902,23.524902 0 0 1 -5.04173,-2.54001 l -6.05573,3.46919 a 29.907628,29.907628 0 0 1 -3.58191,-3.49598 29.907628
 ,29.907628 0 0 1 -2.9233,-4.07194 l 4.27314,-5.44883 a 23.524902,23.524902 0 0 1 -1.75454,-5.36936 l -6.72324,-1.82616 a 29.907628,29.907628 0 0 1 0.73798,-9.9567 l 6.88304,-0.8322 a 23.524902,23.524902 0 0 1 2.54001,-5.04173 l -3.46885,-6.05502 a 29.907628,29.907628 0 0 1 3.49494,-3.58228 29.907628,29.907628 0 0 1 4.07194,-2.92331 l 5.44901,4.27262 a 23.524902,23.524902 0 0 1 5.36988,-1.75436 l 1.82616,-6.72324 a 29.907628,29.907628 0 0 1 9.95599,0.73833 l 0.8329,6.8827 a 23.524902,23.524902 0 0 1 5.04121,2.53983 l 6.055556,-3.46867 a 29.907628,29.907628 0 0 1 3.58227,3.49494 29.907628,29.907628 0 0 1 1.52975,1.98522 44.775326,44.775326 0 0 0 -29.979576,-45.3099 z m -29.93126,7.04882 a 15.745845,15.745845 0 0 0 -20.0396247,9.71017 15.745845,15.745845 0 0 0 9.7101647,20.03962 15.745845,15.745845 0 0 0 20.03963,-9.71016 15.745845,15.745845 0 0 0 -9.71017,-20.03963 z m 59.360536,42.82725 -2.32895,2.97009 a 23.524902,23.524902 0 0 1 0.99553,2.56761 44.775326,44.775326 0 0 0 1.33342,-5.
 5377 z m -19.143966,-2.09047 a 15.745845,15.745845 0 0 0 -14.75675,3.19169 15.745845,15.745845 0 0 0 -1.61104,22.20982 15.745845,15.745845 0 0 0 21.90246,1.85092 44.775326,44.775326 0 0 0 2.25015,-2.25582 15.745845,15.745845 0 0 0 -0.3321,-20.19458 15.745845,15.745845 0 0 0 -7.45272,-4.80203 z"
+       id="path5303"
+       inkscape:connector-curvature="0" />
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/logo2.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/logo2.png b/docs/docs/img/logo2.png
new file mode 100644
index 0000000..959d39e
Binary files /dev/null and b/docs/docs/img/logo2.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/messageLoss.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/messageLoss.png b/docs/docs/img/messageLoss.png
new file mode 100644
index 0000000..80b330a
Binary files /dev/null and b/docs/docs/img/messageLoss.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/netty_transport.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/netty_transport.png b/docs/docs/img/netty_transport.png
new file mode 100644
index 0000000..17d57c3
Binary files /dev/null and b/docs/docs/img/netty_transport.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/replay.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/replay.png b/docs/docs/img/replay.png
new file mode 100644
index 0000000..8bbbc43
Binary files /dev/null and b/docs/docs/img/replay.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/shuffle.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/shuffle.png b/docs/docs/img/shuffle.png
new file mode 100644
index 0000000..40c4a2d
Binary files /dev/null and b/docs/docs/img/shuffle.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/storm_gearpump_cluster.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/storm_gearpump_cluster.png b/docs/docs/img/storm_gearpump_cluster.png
new file mode 100644
index 0000000..d318623
Binary files /dev/null and b/docs/docs/img/storm_gearpump_cluster.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/storm_gearpump_dag.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/storm_gearpump_dag.png b/docs/docs/img/storm_gearpump_dag.png
new file mode 100644
index 0000000..24920f7
Binary files /dev/null and b/docs/docs/img/storm_gearpump_dag.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/submit.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/submit.png b/docs/docs/img/submit.png
new file mode 100644
index 0000000..609c0d7
Binary files /dev/null and b/docs/docs/img/submit.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/submit2.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/submit2.png b/docs/docs/img/submit2.png
new file mode 100644
index 0000000..d3939ee
Binary files /dev/null and b/docs/docs/img/submit2.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/img/through_vs_message_size.png
----------------------------------------------------------------------
diff --git a/docs/docs/img/through_vs_message_size.png b/docs/docs/img/through_vs_message_size.png
new file mode 100644
index 0000000..a98c528
Binary files /dev/null and b/docs/docs/img/through_vs_message_size.png differ

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/docs/index.md b/docs/docs/index.md
new file mode 100644
index 0000000..1cd98b8
--- /dev/null
+++ b/docs/docs/index.md
@@ -0,0 +1,35 @@
+---
+layout: global
+displayTitle: Gearpump Overview
+title: Overview
+description: Gearpump GEARPUMP_VERSION documentation homepage
+---
+
+Gearpump is a real-time big data streaming engine.
+It is inspired by recent advances in the [Akka](http://akka.io/) framework and a desire to improve on existing streaming frameworks.
+Gearpump is event/message based and featured as low latency handling, high performance, exactly once semantics,
+dynamic topology update, [Apache Storm](https://storm.apache.org/) compatibility, etc.
+
+The	name	Gearpump	is	a	reference to	the	engineering term "gear	pump,"	which	is	a	super simple
+pump	that	consists of	only	two	gears,	but	is	very	powerful at	streaming water.
+
+### Gearpump Technical Highlights
+Gearpump's feature set includes:
+
+* Extremely high performance
+* Low latency
+* Configurable message delivery guarantee (at least once, exactly once).
+* Highly extensible
+* Dynamic DAG
+* Storm compatibility
+* Samoa compatibility
+* Both high level and low level API
+
+### Gearpump Performance
+Per initial benchmarks we are able to process 18 million messages/second (100 bytes per message) with a 8ms latency on a 4-node cluster.
+
+![Dashboard](img/dashboard.png)
+
+### Gearpump and Akka
+Gearpump is a 100% Akka based platform. We model big data streaming within the Akka actor hierarchy.
+![Actor Hierarchy](img/actor_hierarchy.png)

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/basic-concepts.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/basic-concepts.md b/docs/docs/introduction/basic-concepts.md
new file mode 100644
index 0000000..a4c49b7
--- /dev/null
+++ b/docs/docs/introduction/basic-concepts.md
@@ -0,0 +1,46 @@
+### System timestamp and Application timestamp
+
+System timestamp denotes the time of backend cluster system. Application timestamp denotes the time at which message is generated. For example, for IoT edge device, the timestamp at which field sensor device creates a message is type of application timestamp, while the timestamp at which that message get received by the backend is type of system time.
+
+### Master, and Worker
+
+Gearpump follow master slave architecture. Every cluster contains one or more Master node, and several worker nodes. Worker node is responsible to manage local resources on single machine, and Master node is responsible to manage global resources of the whole cluster.
+
+![Actor Hierarchy](../img/actor_hierarchy.png)
+
+### Application
+
+Application is what we want to parallel and run on the cluster. There are different application types, for example MapReduce application and streaming application are different application types. Gearpump natively supports Streaming Application types, it also contains several templates to help user to create custom application types, like distributedShell.
+
+### AppMaster and Executor
+
+In runtime, every application instance is represented by a single AppMaster and a list of Executors. AppMaster represents the command and controls center of the Application instance. It communicates with user, master, worker, and executor to get the job done. Each executor is a parallel unit for distributed application. Typically AppMaster and Executor will be started as JVM processes on worker nodes.
+
+### Application Submission Flow
+
+When user submits an application to Master, Master will first find an available worker to start the AppMaster. After AppMaster is started, AppMaster will request Master for more resources (worker) to start executors. The Executor now is only an empty container. After the executors are started, the AppMaster will then distribute real computation tasks to the executor and run them in parallel way.
+
+To submit an application, a Gearpump client specifies a computation defined within a DAG and submits this to an active master. The SubmitApplication message is sent to the Master who then forwards this to an AppManager.
+
+![Submit App](../img/submit.png)
+Figure: User Submit Application
+
+The AppManager locates an available worker and launches an AppMaster in a sub-process JVM of the worker. The AppMaster will then negotiate with the Master for Resource allocation in order to distribute the DAG as defined within the Application. The allocated workers will then launch Executors (new JVMs).
+
+![Launch Executors and Tasks](../img/submit2.png)
+Figure: Launch Executors and Tasks
+
+### Streaming Topology, Processor, and Task
+
+For streaming application type, each application contains a topology, which is a DAG (directed acyclic graph) to describe the data flow. Each node in the DAG is a processor. For example, for word count it contains two processors, Split and Sum. The Split processor splits a line to a list of words, and then the Sum processor summarize the frequency of each word.
+An application is a DAG of processors. Each processor handles messages.
+
+![DAG](../img/dag.png)
+Figure: Processor DAG
+
+### Streaming Task and Partitioner
+
+For streaming application type, Task is the minimum unit of parallelism. In runtime, each Processor is paralleled to a list of tasks, with different tasks running in different executor. You can define Partitioner to denote the data shuffling rule between upstream processor tasks and downstream processor tasks.
+
+![Data Shuffle](../img/shuffle.png)
+Figure: Task Data Shuffling

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/commandline.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/commandline.md b/docs/docs/introduction/commandline.md
new file mode 100644
index 0000000..3dfe885
--- /dev/null
+++ b/docs/docs/introduction/commandline.md
@@ -0,0 +1,84 @@
+The commands can be found at: "bin" folder of Gearpump binary.
+
+**NOTE:** on MS Windows platform, please use window shell gear.bat script instead. bash script doesn't work well in cygwin/mingw.
+
+### Creating an uber-jar
+
+If you use Maven you can have a look [here](https://maven.apache.org/plugins/maven-shade-plugin/) whereas SBT users may find [this](https://github.com/sbt/sbt-assembly) useful.
+
+### Submit an new application
+
+You can use the command `gear` under the bin directory to submit, query and terminate an application:
+
+	:::bash
+	gear app [-namePrefix <application name prefix>] [-executors <number of executors to launch>] [-conf <custom gearpump config file>] -jar xx.jar MainClass <arg1> <arg2> ...
+	
+
+### List all running applications
+To list all running applications:
+
+	:::bash
+	gear info  [-conf <custom gearpump config file>]
+
+
+### Kill a running application
+To kill an application:
+
+	:::bash
+	gear kill -appid <application id>  [-conf <custom gearpump config file>]
+
+
+### Submit a storm application to Gearpump Cluster
+For example, to submit a storm application jar:
+
+	:::bash
+	storm -verbose -config storm.yaml -jar storm-starter-${STORM_VERSION}.jar storm.starter.ExclamationTopology exclamation
+	
+
+[Storm Compatibility Guide](../dev/dev-storm)
+
+### Start Gearpump Cluster on YARN
+To start a Gearpump Cluster on YARN, you can:
+
+	:::bash
+	yarnclient launch -package /usr/lib/gearpump/gearpump-{{SCALA_BINARY_VERSION}}-{{GEARPUMP_VERSION}}.zip
+
+`/usr/lib/gearpump/gearpump-{{SCALA_BINARY_VERSION}}-{{GEARPUMP_VERSION}}.zip` should be available on HDFS.
+
+Please check [YARN Deployment Guide](../deployment/deployment-yarn) for more information.
+
+### Start a local cluster
+Masters and workers will be started in one machine:
+
+	:::bash
+	local
+	
+
+Check [Deployment Guide for Local Cluster](../deployment/deployment-local) for more information.
+
+### Start master daemons
+
+	:::bash
+	master -ip <Ip address> -port <port where this master is hooking>
+
+
+Please check [Deployment for Standalone mode](../deployment/deployment-standalone) for more information.
+
+### Start worker daemons
+
+	:::bash
+	worker
+
+
+Please check [Deployment for Standalone mode](../deployment/deployment-standalone) for more information.
+
+### Start UI server
+
+To start UI server, you can:
+
+	:::bash
+	services  [-master <host:port>]
+
+
+The default username and password is "admin:admin", you can check
+[UI Authentication](../deployment/deployment-ui-authentication) to find how to manage users.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/features.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/features.md b/docs/docs/introduction/features.md
new file mode 100644
index 0000000..4fcb66b
--- /dev/null
+++ b/docs/docs/introduction/features.md
@@ -0,0 +1,67 @@
+### Technical highlights of Gearpump
+
+Gearpump is a high performance, flexible, fault-tolerant, and responsive streaming platform with a lot of nice features, its technical highlights include:
+
+#### Actors everywhere
+
+The Actor model is a concurrency model proposed by Carl Hewitt at 1973. The Actor model is like a micro-service which is cohesive in the inside and isolated from other outside actors. Actors are the cornerstone of Gearpump, they provide facilities to do message passing, error handling, liveliness monitoring. Gearpump uses Actors everywhere; every entity within the cluster that can be treated as a service.
+
+![Actor Hierarchy](../img/actor_hierarchy.png)
+
+#### Exactly once Message Processing
+
+Exactly once is defined as: the effect of a message will be calculated only once in the persisted state and computation errors in the history will not be propagated to future computations.
+
+![Exact Once Semantics](../img/exact.png)
+
+#### Topology DAG DSL
+
+User can submit to Gearpump a computation DAG, which contains a list of nodes and edges, and each node can be parallelized to a set of tasks. Gearpump will then schedule and distribute different tasks in the DAG to different machines automatically. Each task will be started as an actor, which is long running micro-service.
+
+![DAG](../img/dag.png)
+
+#### Flow control
+
+Gearpump has built-in support for flow control. For all message passing between different tasks, the framework will assure the upstream tasks will not flood the downstream tasks.
+![Flow Control](../img/flowcontrol.png)
+
+#### No inherent end to end latency
+
+Gearpump is a message level streaming engine, which means every task in the DAG will process messages immediately upon receiving, and deliver messages to downstream immediately without waiting. Gearpump doesn't do batching when data sourcing.
+
+#### High Performance message passing
+
+By implementing smart batching strategies, Gearpump is extremely effective in transferring small messages. In one test of 4 machines, the whole cluster throughput can reach 18 million messages per second, with message size of 100 bytes.
+![Dashboard](../img/dashboard.png)
+
+#### High availability, No single point of failure
+
+Gearpump has a careful design for high availability. We have considered message loss, worker machine crash, application crash, master crash, brain-split, and have made sure Gearpump recovers when these errors may occur. When there is message loss, the lost message will be replayed; when there is a worker machine crash or application crash, the related computation tasks will be rescheduled on new machines. For master high availability, several master nodes will form a Akka cluster, and CRDTs (conflict free data types) are used to exchange the state, so as long as there is still a quorum, the master will stay functional. When one master node fails, other master nodes in the cluster will take over and state will be recovered.
+
+![HA](../img/ha.png)
+
+#### Dynamic Computation DAG
+
+Gearpump provides a feature which allows the user to dynamically add, remove, or replace a sub graph at runtime, without the need to restart the whole computation topology.
+
+![Dynamic DAG](../img/dynamic.png)
+
+#### Able to handle out of order messages
+
+For a window operation like moving average on a sliding window, it is important to make sure we have received all messages in that time window so that we can get an accurate result, but how do we handle stranglers or late arriving messages? Gearpump solves this problem by tracking the low watermark of timestamp of all messages, so it knows whether we've received all the messages in the time window or not.
+
+![Clock](../img/clock.png)
+
+#### Customizable platform
+
+Different applications have different requirements related to performance metrics, some may want higher throughput, some may require strong eventual data consistency; and different applications have different resource requirements profiles, some may demand high CPU performance, some may require data locality. Gearpump meets these requirements by allowing the user to arbitrate between different performance metrics and define customized resource scheduling strategies.
+
+#### Built-in Dashboard UI
+
+Gearpump has a built-in dashboard UI to manage the cluster and visualize the applications. The UI uses REST calls to connect with backend, so it is easy to embed the UI within other dashboards.
+
+![Dashboard](../img/dashboard.gif)
+
+#### Data connectors for Kafka and HDFS
+
+Gearpump has built-in data connectors for Kafka and HDFS. For the Kafka connector, we support message replay from a specified timestamp.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/gearpump-internals.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/gearpump-internals.md b/docs/docs/introduction/gearpump-internals.md
new file mode 100644
index 0000000..bc7e6bf
--- /dev/null
+++ b/docs/docs/introduction/gearpump-internals.md
@@ -0,0 +1,228 @@
+### Actor Hierarchy?
+
+![Actor Hierarchy](../img/actor_hierarchy.png)
+
+Everything in the diagram is an actor; they fall into two categories, Cluster Actors and Application Actors.
+
+#### Cluster Actors
+
+  **Worker**: Maps to a physical worker machine. It is responsible for managing resources and report metrics on that machine.
+
+  **Master**: Heart of the cluster, which manages workers, resources, and applications. The main function is delegated to three child actors, App Manager, Worker Manager, and Resource Scheduler.
+
+#### Application Actors:
+
+  **AppMaster**: Responsible to schedule the tasks to workers and manage the state of the application. Different applications have different AppMaster instances and are isolated.
+
+  **Executor**: Child of AppMaster, represents a JVM process. Its job is to manage the life cycle of tasks and recover the tasks in case of failure.
+
+  **Task**: Child of Executor, does the real job. Every task actor has a global unique address. One task actor can send data to any other task actors. This gives us great flexibility of how the computation DAG is distributed.
+
+  All actors in the graph are weaved together with actor supervision, and actor watching and every error is handled properly via supervisors. In a master, a risky job is isolated and delegated to child actors, so it's more robust. In the application, an extra intermediate layer "Executor" is created so that we can do fine-grained and fast recovery in case of task failure. A master watches the lifecycle of AppMaster and worker to handle the failures, but the life cycle of Worker and AppMaster are not bound to a Master Actor by supervision, so that Master node can fail independently.  Several Master Actors form an Akka cluster, the Master state is exchanged using the Gossip protocol in a conflict-free consistent way so that there is no single point of failure. With this hierarchy design, we are able to achieve high availability.
+
+### Application Clock and Global Clock Service
+
+Global clock service will track the minimum time stamp of all pending messages in the system. Every task will update its own minimum-clock to global clock service; the minimum-clock of task is decided by the minimum of:
+
+  - Minimum time stamp of all pending messages in the inbox.
+  - Minimum time stamp of all un-acked outgoing messages. When there is message loss, the minimum clock will not advance.
+  - Minimum clock of all task states. If the state is accumulated by a lot of input messages, then the clock value is decided by the oldest message's timestamp. The state clock will advance by doing snapshots to persistent storage or by fading out the effect of old messages.
+
+![Clock](../img/clock.png)
+
+The global clock service will keep track of all task minimum clocks effectively and maintain a global view of minimum clock. The global minimum clock value is monotonically increasing; it means that all source messages before this clock value have been processed. If there is message loss or task crash, the global minimum clock will stop.
+
+### How do we optimize the message passing performance?
+
+For streaming application, message passing performance is extremely important. For example, one streaming platform may need to process millions of messages per second with millisecond level latency. High throughput and low latency is not that easy to achieve. There are a number of challenges:
+
+#### First Challenge: Network is not efficient for small messages
+
+In streaming, typical message size is very small, usually less than 100 bytes per message, like the floating car GPS data. But network efficiency is very bad when transferring small messages. As you can see in below diagram, when message size is 50 bytes, it can only use 20% bandwidth. How to improve the throughput?
+
+![Throughput vs. Message Size](../img/through_vs_message_size.png)
+
+#### Second Challenge: Message overhead is too big
+
+For each message sent between two actors, it contains sender and receiver actor path. When sending over the wire, the overhead of this ActorPath is not trivial. For example, the below actor path takes more than 200 bytes.
+
+	:::bash
+	akka.tcp://system1@192.168.1.53:51582/remote/akka.tcp/2120193a-e10b-474e-bccb-8ebc4b3a0247@192.168.1.53:48948/remote/akka.tcp/system2@192.168.1.54:43676/user/master/Worker1/app_0_executor_0/group_1_task_0#-768886794
+
+
+#### How do we solve this?
+
+We implement a custom Netty transportation layer with Akka extension. In the below diagram, Netty Client will translate ActorPath to TaskId, and Netty Server will translate it back. Only TaskId will be passed on wire, it is only about 10 bytes, the overhead is minimized. Different Netty Client Actors are isolated; they will not block each other.
+
+![Netty Transport](../img/netty_transport.png)
+
+For performance, effective batching is really the key! We group multiple messages to a single batch and send it on the wire. The batch size is not fixed; it is adjusted dynamically based on network status. If the network is available, we will flush pending messages immediately without waiting; otherwise we will put the message in a batch and trigger a timer to flush the batch later.
+
+### How do we do flow Control?
+
+Without flow control, one task can easily flood another task with too many messages, causing out of memory error. Typical flow control will use a TCP-like sliding window, so that source and target can run concurrently without blocking each other.
+
+![Flow Control](../img/flow_control.png)
+Figure: Flow control, each task is "star" connected to input tasks and output tasks
+
+The difficult part for our problem is that each task can have multiple input tasks and output tasks. The input and output must be geared together so that the back pressure can be properly propagated from downstream to upstream. The flow control also needs to consider failures, and it needs to be able to recover when there is message loss.
+Another challenge is that the overhead of flow control messages can be big. If we ack every message, there will be huge amount of acked messages in the system, degrading streaming performance. The approach we adopted is to use explicit AckRequest message. The target tasks will only ack back when they receive the AckRequest message, and the source will only send AckRequest when it feels necessary. With this approach, we can largely reduce the overhead.
+
+### How do we detect message loss?
+
+For example, for web ads, we may charge for every click, we don't want to miscount.  The streaming platform needs to effectively track what messages have been lost, and recover as fast as possible.
+
+![Message Loss](../img/messageLoss.png)
+Figure: Message Loss Detection
+
+We use the flow control message AckRequest and Ack to detect message loss. The target task will count how many messages has been received since last AckRequest, and ack the count back to source task. The source task will check the count and find message loss.
+This is just an illustration, the real case is more difficulty, we need to handle zombie tasks, and in-the-fly stale messages.
+
+### How Gearpump know what messages to replay?
+
+In some applications, a message cannot be lost, and must be replayed. For example, during the money transfer, the bank will SMS us the verification code. If that message is lost, the system must replay it so that money transfer can continue. We made the decision to use **source end message storage** and **time stamp based replay**.
+
+![Replay](../img/replay.png)
+Figure: Replay with Source End Message Store
+
+Every message is immutable, and tagged with a timestamp. We have an assumption that the timestamp is approximately incremental (allow small ratio message disorder).
+
+We assume the message is coming from a replay-able source, like Kafka queue; otherwise the message will be stored at customizable source end "message store". When the source task sends the message downstream, the timestamp and offset of the message is also check-pointed to offset-timestamp storage periodically. During recovery, the system will first retrieve the right time stamp and offset from the offset-timestamp storage, then it will replay the message store from that time stamp and offset. A Timestamp Filter will filter out old messages in case the message in message store is not strictly time-ordered.
+
+### Master High Availability
+
+In a distributed streaming system, any part can fail. The system must stay responsive and do recovery in case of errors.
+
+![HA](../img/ha.png)
+Figure: Master High Availability
+
+We use Akka clustering to implement the Master high availability. The cluster consists of several master nodes, but no worker nodes. With clustering facilities, we can easily detect and handle the failure of master node crash. The master state is replicated on all master nodes with the Typesafe akka-data-replication  library, when one master node crashes, another standby master will read the master state and take over. The master state contains the submission data of all applications. If one application dies, a master can use that state to recover that application. CRDT LwwMap  is used to represent the state; it is a hash map that can converge on distributed nodes without conflict. To have strong data consistency, the state read and write must happen on a quorum of master nodes.
+
+### How we do handle failures?
+
+With Akka's powerful actor supervision, we can implement a resilient system relatively easy. In Gearpump, different applications have a different AppMaster instance, they are totally isolated from each other. For each application, there is a supervision tree, AppMaster->Executor->Task. With this supervision hierarchy, we can free ourselves from the headache of zombie process, for example if AppMaster is down, Akka supervisor will ensure the whole tree is shutting down.
+
+There are multiple possible failure scenarios
+
+![Failures](../img/failures.png)
+Figure: Possible Failure Scenarios and Error Supervision Hierarchy
+
+#### What happens when the Master crashes?
+
+In case of a master crash, other standby masters will be notified, they will resume the master state, and take over control. Worker and AppMaster will also be notified, They will trigger a process to find the new active master, until the resolution complete. If AppMaster or Worker cannot resolve a new Master in a time out, they will make suicide and kill themselves.
+
+#### What happens when a worker crashes?
+
+In case of a worker crash, the Master will get notified and stop scheduling new computation to this worker. All supervised executors on current worker will be killed, AppMaster can treat it as recovery of executor crash like [What happen when an executor crashes?](#what-happen-when-an-executor-crashes)
+
+#### What happens when the AppMaster crashes?
+
+If an AppMaster crashes, Master will schedule a new resource to create a new AppMaster Instance elsewhere, and then the AppMaster will handle the recovery inside the application. For streaming, it will recover the latest min clock and other state from disk, request resources from master to start executors, and restart the tasks with recovered min clock.
+
+#### What happen when an executor crashes?
+
+If an executor crashes, its supervisor AppMaster will get notified, and request a new resource from the active master to start a new executor, to run the tasks which were located on the crashed executor.
+
+#### What happen when tasks crash?
+
+If a task throws an exception, its supervisor executor will restart that Task.
+
+When "at least once" message delivery is enabled, it will trigger the message replaying in the case of message loss. First AppMaster will read the latest minimum clock from the global clock service(or clock storage if the clock service crashes), then AppMaster will restart all the task actors to get a fresh task state, then the source end tasks will replay messages from that minimum clock.
+
+### How does "exactly-once" message delivery work?
+
+For some applications, it is extremely important to do "exactly once" message delivery. For example, for a real-time billing system, we will not want to bill the customer twice. The goal of "exactly once" message delivery is to make sure:
+  The error doesn't accumulate, today's error will not be accumulated to tomorrow.
+  Transparent to application developer
+We use global clock to synchronize the distributed transactions. We assume every message from the data source will have a unique timestamp, the timestamp can be a part of the message body, or can be attached later with system clock when the message is injected into the streaming system. With this global synchronized clock, we can coordinate all tasks to checkpoint at same timestamp.
+
+![Checkpoint](../img/checkpointing.png)
+Figure: Checkpointing and Exactly-Once Message delivery
+
+Workflow to do state checkpointing:
+
+1. The coordinator asks the streaming system to do checkpoint at timestamp Tc.
+2. For each application task, it will maintain two states, checkpoint state and current state. Checkpoint state only contains information before timestamp Tc. Current state contains all information.
+3. When global minimum clock is larger than Tc, it means all messages older than Tc has been processed; the checkpoint state will no longer change, so we will then persist the checkpoint state to storage safely.
+4. When there is message loss, we will start the recovery process.
+5. To recover, load the latest checkpoint state from store, and then use it to restore the application status.
+6. Data source replays messages from the checkpoint timestamp.
+
+The checkpoint interval is determined by global clock service dynamically. Each data source will track the max timestamp of input messages. Upon receiving min clock updates, the data source will report the time delta back to global clock service. The max time delta is the upper bound of the application state timespan. The checkpoint interval is bigger than max delta time:
+
+![Checkpoint Equation](../img/checkpoint_equation.png)
+
+![Checkpointing Interval](../img/checkpointing_interval.png)
+Figure: How to determine Checkpoint Interval
+
+After the checkpoint interval is notified to tasks by global clock service, each task will calculate its next checkpoint timestamp autonomously without global synchronization.
+
+![Checkpoint Interval Equation](../img/checkpoint_interval_equation.png)
+
+For each task, it contains two states, checkpoint state and current state. The code to update the state is shown in listing below.
+
+	:::python
+	TaskState(stateStore, initialTimeStamp):
+	  currentState = stateStore.load(initialTimeStamp)
+	  checkpointState = currentState.clone
+	  checkpointTimestamp = nextCheckpointTimeStamp(initialTimeStamp)
+	onMessage(msg):
+	  if (msg.timestamp < checkpointTimestamp):
+	    checkpointState.updateMessage(msg)
+	  currentState.updateMessage(msg)  
+	  maxClock = max(maxClock, msg.timeStamp)
+	
+	onMinClock(minClock):
+	  if (minClock > checkpointTimestamp):
+	    stateStore.persist(checkpointState)
+	    checkpointTimeStamp = nextCheckpointTimeStamp(maxClock)
+	    checkpointState = currentState.clone
+	
+	onNewCheckpointInterval(newStep):
+	  step = newStep  
+	nextCheckpointTimeStamp(timestamp):
+	  checkpointTimestamp = (1 + timestamp/step) * step
+	
+
+List 1: Task Transactional State Implementation
+
+### What is dynamic graph, and how it works?
+
+The DAG can be modified dynamically. We want to be able to dynamically add, remove, and replace a sub-graph.
+
+![Dynamic DAG](../img/dynamic.png)
+Figure: Dynamic Graph, Attach, Replace, and Remove
+
+## At least once message delivery and Kafka
+
+The Kafka source example project and tutorials can be found at:
+- [Kafka connector example project](https://github.com/apache/incubator-gearpump/tree/master/examples/streaming/kafka)
+- [Connect with Kafka source](../dev/dev-connectors)
+
+In this doc, we will talk about how the at least once message delivery works.
+
+We will use the WordCount example of [source tree](https://github.com/apache/incubator-gearpump/tree/master/examples/streaming/kafka)  to illustrate.
+
+### How the kafka WordCount DAG looks like:
+
+It contains three processors:
+![Kafka WordCount](../img/kafka_wordcount.png)
+
+- KafkaStreamProducer(or KafkaSource) will read message from kafka queue.
+- Split will split lines to words
+- Sum will summarize the words to get a count for each word.
+
+### How to read data from Kafka
+
+We use KafkaSource, please check [Connect with Kafka source](../dev/dev-connectors) for the introduction.
+
+Please note that we have set a startTimestamp for the KafkaSource, which means KafkaSource will read from Kafka queue starting from messages whose timestamp is near startTimestamp.
+
+### What happen where there is Task crash or message loss?
+When there is message loss, the AppMaster will first pause the global clock service so that the global minimum timestamp no longer change, then it will restart the Kafka source tasks. Upon restart, Kafka Source will start to replay. It will first read the global minimum timestamp from AppMaster, and start to read message from that timestamp.
+
+### What method KafkaSource used to read messages from a start timestamp? As we know Kafka queue doesn't expose the timestamp information.
+
+Kafka queue only expose the offset information for each partition. What KafkaSource do is to maintain its own mapping from Kafka offset to  Application timestamp, so that we can map from a application timestamp to a Kafka offset, and replay Kafka messages from that Kafka offset.
+
+The mapping between Application timestamp with Kafka offset is stored in a distributed file system or as a Kafka topic.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/message-delivery.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/message-delivery.md b/docs/docs/introduction/message-delivery.md
new file mode 100644
index 0000000..1ffeb9e
--- /dev/null
+++ b/docs/docs/introduction/message-delivery.md
@@ -0,0 +1,47 @@
+## What is At Least Once Message Delivery?
+
+Messages could be lost on delivery due to network partitions. **At Least Once Message Delivery** (at least once) means the lost messages are delivered one or more times such that at least one is processed and acknowledged by the whole flow. 
+
+Gearpump guarantees at least once for any source that is able to replay message from a past timestamp. In Gearpump, each message is tagged with a timestamp, and the system tracks the minimum timestamp of all pending messages (the global minimum clock). On message loss, application will be restarted to the global minimum clock. Since the source is able to replay from the global minimum clock, all pending messages before the restart will be replayed. Gearpump calls that kind of source `TimeReplayableSource` and already provides a built in
+[KafkaSource](gearpump-internals#at-least-once-message-delivery-and-kafka). With the KafkaSource to ingest data into Gearpump, users are guaranteed at least once message delivery.
+
+## What is Exactly Once Message Delivery?
+
+At least once delivery doesn't guarantee the correctness of the application result. For instance,  for a task keeping the count of received messages, there could be overcount with duplicated messages and the count is lost on task failure.
+ In that case, **Exactly Once Message Delivery** (exactly once) is required, where state is updated by a message exactly once. This further requires that duplicated messages are filtered out and in-memory states are persisted.
+
+Users are guaranteed exactly once in Gearpump if they use both a `TimeReplayableSource` to ingest data and the Persistent API to manage their in memory states. With the Persistent API, user state is periodically checkpointed by the system to a persistent store (e.g HDFS) along with its checkpointed time. Gearpump tracks the global minimum checkpoint timestamp of all pending states (global minimum checkpoint clock), which is persisted as well. On application restart, the system restores states at the global minimum checkpoint clock and source replays messages from that clock. This ensures that a message updates all states exactly once.
+
+### Persistent API
+Persistent API consists of `PersistentTask` and `PersistentState`.
+
+Here is an example of using them to keep count of incoming messages.
+
+	:::scala
+	class CountProcessor(taskContext: TaskContext, conf: UserConfig)
+  	  extends PersistentTask[Long](taskContext, conf) {
+
+  	  override def persistentState: PersistentState[Long] = {
+        import com.twitter.algebird.Monoid.longMonoid
+        new NonWindowState[Long](new AlgebirdMonoid(longMonoid), new ChillSerializer[Long])
+      }
+
+      override def processMessage(state: PersistentState[Long], message: Message): Unit = {
+        state.update(message.timestamp, 1L)
+      }
+    }
+
+   
+The `CountProcessor` creates a customized `PersistentState` which will be managed by `PersistentTask` and overrides the `processMessage` method to define how the state is updated on a new message (each new message counts as `1`, which is added to the existing value)
+
+Gearpump has already offered two types of states
+ 
+1. NonWindowState - state with no time or other boundary
+2. WindowState - each state is bounded by a time window
+
+They are intended for states that satisfy monoid laws.
+
+1. has binary associative operation, like `+`  
+2. has an identity element, like `0`
+
+In the above example, we make use of the `longMonoid` from [Twitter's Algebird](https://github.com/twitter/algebird) library which provides a bunch of useful monoids. 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/performance-report.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/performance-report.md b/docs/docs/introduction/performance-report.md
new file mode 100644
index 0000000..e4254d1
--- /dev/null
+++ b/docs/docs/introduction/performance-report.md
@@ -0,0 +1,34 @@
+# Performance Evaluation
+
+To illustrate the performance of Gearpump, we mainly focused on two aspects, throughput and latency, using a micro benchmark called SOL (an example in the Gearpump package) whose topology is quite simple. SOLStreamProducer delivers messages to SOLStreamProcessor constantly and SOLStreamProcessor does nothing. We set up a 4-nodes cluster with 10GbE network and each node's hardware is briefly shown as follows:
+
+Processor: 32 core Intel(R) Xeon(R) CPU E5-2690 2.90GHz
+Memory: 64GB
+
+## Throughput
+
+We tried to explore the upper bound of the throughput, after launching 48 SOLStreamProducer and 48 SOLStreamProcessor the Figure below shows that the whole throughput of the cluster can reach about 18 million messages/second(100 bytes per message)
+
+## Latency
+
+When we transfer message at the max throughput above, the average latency between two tasks is 8ms.
+
+## Fault Recovery time
+
+When the corruption is detected, for example the Executor is down, Gearpump will reallocate the resource and restart the application. It takes about 10 seconds to recover the application.
+
+![Dashboard](../img/dashboard.png)
+
+## How to setup the benchmark environment?
+
+### Prepare the env
+
+1). Set up a 4-nodes Gearpump cluster with 10GbE network which have 4 Workers on each node. In our test environment, each node has 64GB memory and Intel(R) Xeon(R) 32-core processor E5-2690 2.90GHz. Make sure the metrics is enabled in Gearpump.
+
+2). Submit a SOL application with 48 StreamProducers and 48 StreamProcessors:
+
+    :::bash
+    bin/gear app -jar ./examples/sol-{{SCALA_BINARY_VERSION}}-{{GEARPUMP_VERSION}}-assembly.jar -streamProducer 48 -streamProcessor 48
+
+
+3). Launch Gearpump's dashboard and browser http://$HOST:8090/, switch to the Applications tab and you can see the detail information of your application. The HOST should be the node runs dashboard.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/docs/introduction/submit-your-1st-application.md
----------------------------------------------------------------------
diff --git a/docs/docs/introduction/submit-your-1st-application.md b/docs/docs/introduction/submit-your-1st-application.md
new file mode 100644
index 0000000..21cfaf2
--- /dev/null
+++ b/docs/docs/introduction/submit-your-1st-application.md
@@ -0,0 +1,39 @@
+Before you can submit and run your first Gearpump application, you will need a running Gearpump service.
+There are multiple ways to run Gearpump [Local mode](../deployment/deployment-local), [Standalone mode](../deployment/deployment-standalone), [YARN mode](../deployment/deployment-yarn) or [Docker mode](../deployment/deployment-docker).
+
+The easiest way is to run Gearpump in [Local mode](../deployment/deployment-local).
+Any Linux, MacOSX or Windows desktop can be used with zero configuration.
+
+In the example below, we assume your are running in [Local mode](../deployment/deployment-local).
+If you running Gearpump in one of the other modes, you will need to configure the Gearpump client to
+connect to the Gearpump service by setting the `gear.conf` configuration path in classpath.
+Within this file, you will need to change the parameter `gearpump.cluster.masters` to the correct Gearpump master(s).
+See [Configuration](../deployment/deployment-configuration) for details.
+
+## Steps to submit your first Application
+
+### Step 1: Submit application
+After the cluster is started, you can submit an example wordcount application to the cluster
+
+Open another shell,
+
+	:::bash
+	### To run WordCount example
+	bin/gear app -jar examples/wordcount-{{SCALA_BINARY_VERSION}}-{{GEARPUMP_VERSION}}-assembly.jar org.apache.gearpump.streaming.examples.wordcount.WordCount
+
+
+###  Step 2: Congratulations, you've submitted your first application.
+
+To view the application status and metrics, start the Web UI services, and browse to [http://127.0.0.1:8090](http://127.0.0.1:8090) to check the status.
+The default username and password is "admin:admin", you can check
+[UI Authentication](../deployment/deployment-ui-authentication) to find how to manage users.
+
+![Dashboard](../img/dashboard.gif)
+
+**NOTE:** the UI port setting can be defined in configuration, please check section [Configuration](../deployment/deployment-configuration).
+
+## A quick Look at the Web UI
+TBD
+
+## Other Application Examples
+Besides wordcount, there are several other example applications. Please check the source tree examples/ for detail information.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/faq.md
----------------------------------------------------------------------
diff --git a/docs/faq.md b/docs/faq.md
deleted file mode 100644
index 39bb7f2..0000000
--- a/docs/faq.md
+++ /dev/null
@@ -1,39 +0,0 @@
----
-layout: global
-displayTitle: Frequently Asked Questions
-title: faq
-description: Gearpump Frequently Asked Questions
----
-
-##### What's Relation between Gearpump and YARN?
-Gearpump can run on top of YARN as a YARN application. Gearpump's ApplicationMaster provides the application management, deployment and scheduling of DAGs after arbitrating and receiving container resources from YARN
-
-##### Relation with Storm and Spark Streaming
-Storm and spark streaming are proven platforms, and there are many production deployments. Compared with them, Gearpump is less proven and there is no production deployment yet. However, there is no single platform that can cover every use case; Gearpump has its own +1 points in some special fields. As an instance, for IoT use cases, Gearpump may be considered convenient because the topology can be deployed to edge device with feature of location transparency. For another example, when users want to upgrade the application online without service interruption, Gearpump may be suitable as it can dynamically modify the computation DAG on the fly.
-
-##### What does Gearpump mean?
-The name Gearpump is a reference to the engineering term "Gear Pump", which is a super simple pump that consists of only two gears, but is very powerful at streaming water from left to right.
-
-##### Why not using akka persistence to store the checkpoint file?
-1. We only checkpoint file to disk when necessary.(not record level)
-2. We have custom checkpoint file format
-
-##### Have you considered the akka stream API for the high level DSL?
-We are looking into a hands of candidates for what a good DSL should be. Akka stream API is one of the candidates.
-
-##### Why wrapping the Task, instead of using the Actor interface directly?
-
-1. It is easier to conduct Unit test
-2. We have custom logic and messages to ensure the data consistency, like flow control and message loss detection.
-3. As the Gearpump interface evolves rapidly, for now, we want to be conservative in exposing more powerful functions so that we don't tie our hands for future refactoring. It lets us feel safe.
-
-##### Why does my task has extremely high message latency (e.g. 10 seconds) ?
-
-Please check whether you are doing blocking jobs (e.g. sleep, IO) in your task. By default, all tasks in an executor share a thread pool. The blocking tasks could use up all the threads while other tasks don't get a chance to run. In that case, you can set `gearpump.task-dispatcher` to `"gearpump.single-thread-dispatcher"` in `gear.conf` such that a unique thread is dedicated to each task.
-
-Generally, we recommend use the default `share-thread-pool-dispatcher` which has better performance and only turn to the `single-thread-dispatcher` when you have to.
-
-##### Why can't I open Dashboard even if the Services process has been launched successfully ?
-
-By default, our Services process binds to a local **IPv6 port**. It's possible that another process on your system has already taken up the same **IPv4 port**. You may check by `lsof -i -P | grep -i "Listen"` if your system is Unix/Linux. 
-

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/features.md
----------------------------------------------------------------------
diff --git a/docs/features.md b/docs/features.md
deleted file mode 100644
index 210ac7a..0000000
--- a/docs/features.md
+++ /dev/null
@@ -1,74 +0,0 @@
----
-layout: global
-displayTitle: Gearpump Technical Highlights
-title: highlights
-description: Gearpump Technical Highlights
----
-
-### Technical highlights of Gearpump
-
-Gearpump is a high performance, flexible, fault-tolerant, and responsive streaming platform with a lot of nice features, its technical highlights include:
-
-#### Actors everywhere
-
-The Actor model is a concurrency model proposed by Carl Hewitt at 1973. The Actor model is like a micro-service which is cohesive in the inside and isolated from other outside actors. Actors are the cornerstone of Gearpump, they provide facilities to do message passing, error handling, liveliness monitoring. Gearpump uses Actors everywhere; every entity within the cluster that can be treated as a service.
-
-![Actor Hierarchy](img/actor_hierarchy.png)
-
-#### Exactly once Message Processing
-
-Exactly once is defined as: the effect of a message will be calculated only once in the persisted state and computation errors in the history will not be propagated to future computations.
-
-![Exact Once Semantics](img/exact.png)
-
-#### Topology DAG DSL
-
-User can submit to Gearpump a computation DAG, which contains a list of nodes and edges, and each node can be parallelized to a set of tasks. Gearpump will then schedule and distribute different tasks in the DAG to different machines automatically. Each task will be started as an actor, which is long running micro-service.
-
-![DAG](img/dag.png)
-
-#### Flow control
-
-Gearpump has built-in support for flow control. For all message passing between different tasks, the framework will assure the upstream tasks will not flood the downstream tasks.
-![Flow Control](img/flowcontrol.png)
-
-#### No inherent end to end latency
-
-Gearpump is a message level streaming engine, which means every task in the DAG will process messages immediately upon receiving, and deliver messages to downstream immediately without waiting. Gearpump doesn't do batching when data sourcing.
-
-#### High Performance message passing
-
-By implementing smart batching strategies, Gearpump is extremely effective in transferring small messages. In one test of 4 machines, the whole cluster throughput can reach 18 million messages per second, with message size of 100 bytes.
-![Dashboard](img/dashboard.png)
-
-#### High availability, No single point of failure
-
-Gearpump has a careful design for high availability. We have considered message loss, worker machine crash, application crash, master crash, brain-split, and have made sure Gearpump recovers when these errors may occur. When there is message loss, the lost message will be replayed; when there is a worker machine crash or application crash, the related computation tasks will be rescheduled on new machines. For master high availability, several master nodes will form a Akka cluster, and CRDTs (conflict free data types) are used to exchange the state, so as long as there is still a quorum, the master will stay functional. When one master node fails, other master nodes in the cluster will take over and state will be recovered.
-
-![HA](img/ha.png)
-
-#### Dynamic Computation DAG
-
-Gearpump provides a feature which allows the user to dynamically add, remove, or replace a sub graph at runtime, without the need to restart the whole computation topology.
-
-![Dynamic DAG](img/dynamic.png)
-
-#### Able to handle out of order messages
-
-For a window operation like moving average on a sliding window, it is important to make sure we have received all messages in that time window so that we can get an accurate result, but how do we handle stranglers or late arriving messages? Gearpump solves this problem by tracking the low watermark of timestamp of all messages, so it knows whether we've received all the messages in the time window or not.
-
-![Clock](img/clock.png)
-
-#### Customizable platform
-
-Different applications have different requirements related to performance metrics, some may want higher throughput, some may require strong eventual data consistency; and different applications have different resource requirements profiles, some may demand high CPU performance, some may require data locality. Gearpump meets these requirements by allowing the user to arbitrate between different performance metrics and define customized resource scheduling strategies.
-
-#### Built-in Dashboard UI
-
-Gearpump has a built-in dashboard UI to manage the cluster and visualize the applications. The UI uses REST calls to connect with backend, so it is easy to embed the UI within other dashboards.
-
-![Dashboard](img/dashboard.gif)
-
-#### Data connectors for Kafka and HDFS
-
-Gearpump has built-in data connectors for Kafka and HDFS. For the Kafka connector, we support message replay from a specified timestamp.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/761e04c6/docs/fonts/glyphicons-halflings-regular.eot
----------------------------------------------------------------------
diff --git a/docs/fonts/glyphicons-halflings-regular.eot b/docs/fonts/glyphicons-halflings-regular.eot
deleted file mode 100644
index b93a495..0000000
Binary files a/docs/fonts/glyphicons-halflings-regular.eot and /dev/null differ