You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2016/10/26 20:07:30 UTC

[1/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 505d48ac2 -> 6cb8a436c


http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/spark.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/spark.adoc b/src/main/asciidoc/_chapters/spark.adoc
new file mode 100644
index 0000000..774d137
--- /dev/null
+++ b/src/main/asciidoc/_chapters/spark.adoc
@@ -0,0 +1,690 @@
+////
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ . . http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+////
+
+[[spark]]
+= HBase and Spark
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+link:http://spark.apache.org/[Apache Spark] is a software framework that is used
+to process data in memory in a distributed manner, and is replacing MapReduce in
+many use cases.
+
+Spark itself is out of scope of this document, please refer to the Spark site for
+more information on the Spark project and subprojects. This document will focus
+on 4 main interaction points between Spark and HBase. Those interaction points are:
+
+Basic Spark::
+  The ability to have an HBase Connection at any point in your Spark DAG.
+Spark Streaming::
+  The ability to have an HBase Connection at any point in your Spark Streaming
+  application.
+Spark Bulk Load::
+  The ability to write directly to HBase HFiles for bulk insertion into HBase
+SparkSQL/DataFrames::
+  The ability to write SparkSQL that draws on tables that are represented in HBase.
+
+The following sections will walk through examples of all these interaction points.
+
+== Basic Spark
+
+This section discusses Spark HBase integration at the lowest and simplest levels.
+All the other interaction points are built upon the concepts that will be described
+here.
+
+At the root of all Spark and HBase integration is the HBaseContext. The HBaseContext
+takes in HBase configurations and pushes them to the Spark executors. This allows
+us to have an HBase Connection per Spark Executor in a static location.
+
+For reference, Spark Executors can be on the same nodes as the Region Servers or
+on different nodes there is no dependence of co-location. Think of every Spark
+Executor as a multi-threaded client application. This allows any Spark Tasks
+running on the executors to access the shared Connection object.
+
+.HBaseContext Usage Example
+====
+
+This example shows how HBaseContext can be used to do a `foreachPartition` on a RDD
+in Scala:
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+...
+
+val hbaseContext = new HBaseContext(sc, config)
+
+rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
+ val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1"))
+ it.foreach((putRecord) => {
+. val put = new Put(putRecord._1)
+. putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
+. bufferedMutator.mutate(put)
+ })
+ bufferedMutator.flush()
+ bufferedMutator.close()
+})
+----
+
+Here is the same example implemented in Java:
+
+[source, java]
+----
+JavaSparkContext jsc = new JavaSparkContext(sparkConf);
+
+try {
+  List<byte[]> list = new ArrayList<>();
+  list.add(Bytes.toBytes("1"));
+  ...
+  list.add(Bytes.toBytes("5"));
+
+  JavaRDD<byte[]> rdd = jsc.parallelize(list);
+  Configuration conf = HBaseConfiguration.create();
+
+  JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
+
+  hbaseContext.foreachPartition(rdd,
+      new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() {
+   public void call(Tuple2<Iterator<byte[]>, Connection> t)
+        throws Exception {
+    Table table = t._2().getTable(TableName.valueOf(tableName));
+    BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
+    while (t._1().hasNext()) {
+      byte[] b = t._1().next();
+      Result r = table.get(new Get(b));
+      if (r.getExists()) {
+       mutator.mutate(new Put(b));
+      }
+    }
+
+    mutator.flush();
+    mutator.close();
+    table.close();
+   }
+  });
+} finally {
+  jsc.stop();
+}
+----
+====
+
+All functionality between Spark and HBase will be supported both in Scala and in
+Java, with the exception of SparkSQL which will support any language that is
+supported by Spark. For the remaining of this documentation we will focus on
+Scala examples for now.
+
+The examples above illustrate how to do a foreachPartition with a connection. A
+number of other Spark base functions  are supported out of the box:
+
+// tag::spark_base_functions[]
+`bulkPut`:: For massively parallel sending of puts to HBase
+`bulkDelete`:: For massively parallel sending of deletes to HBase
+`bulkGet`:: For massively parallel sending of gets to HBase to create a new RDD
+`mapPartition`:: To do a Spark Map function with a Connection object to allow full
+access to HBase
+`hBaseRDD`:: To simplify a distributed scan to create a RDD
+// end::spark_base_functions[]
+
+For examples of all these functionalities, see the HBase-Spark Module.
+
+== Spark Streaming
+http://spark.apache.org/streaming/[Spark Streaming] is a micro batching stream
+processing framework built on top of Spark. HBase and Spark Streaming make great
+companions in that HBase can help serve the following benefits alongside Spark
+Streaming.
+
+* A place to grab reference data or profile data on the fly
+* A place to store counts or aggregates in a way that supports Spark Streaming
+promise of _only once processing_.
+
+The HBase-Spark module\u2019s integration points with Spark Streaming are similar to
+its normal Spark integration points, in that the following commands are possible
+straight off a Spark Streaming DStream.
+
+include::spark.adoc[tags=spark_base_functions]
+
+.`bulkPut` Example with DStreams
+====
+
+Below is an example of bulkPut with DStreams. It is very close in feel to the RDD
+bulk put.
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+val hbaseContext = new HBaseContext(sc, config)
+val ssc = new StreamingContext(sc, Milliseconds(200))
+
+val rdd1 = ...
+val rdd2 = ...
+
+val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
+    Array[Byte], Array[Byte])])]]()
+
+queue += rdd1
+queue += rdd2
+
+val dStream = ssc.queueStream(queue)
+
+dStream.hbaseBulkPut(
+  hbaseContext,
+  TableName.valueOf(tableName),
+  (putRecord) => {
+   val put = new Put(putRecord._1)
+   putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
+   put
+  })
+----
+
+There are three inputs to the `hbaseBulkPut` function.
+. The hbaseContext that carries the configuration boardcast information link us
+to the HBase Connections in the executors
+. The table name of the table we are putting data into
+. A function that will convert a record in the DStream into an HBase Put object.
+====
+
+== Bulk Load
+
+There are two options for bulk loading data into HBase with Spark.  There is the
+basic bulk load functionality that will work for cases where your rows have
+millions of columns and cases where your columns are not consolidated and
+partitions before the on the map side of the Spark bulk load process.
+
+There is also a thin record bulk load option with Spark, this second option is
+designed for tables that have less then 10k columns per row.  The advantage
+of this second option is higher throughput and less over all load on the Spark
+shuffle operation.
+
+Both implementations work more or less like the MapReduce bulk load process in
+that a partitioner partitions the rowkeys based on region splits and
+the row keys are sent to the reducers in order, so that HFiles can be written
+out directly from the reduce phase.
+
+In Spark terms, the bulk load will be implemented around a the Spark
+`repartitionAndSortWithinPartitions` followed by a Spark `foreachPartition`.
+
+First lets look at an example of using the basic bulk load functionality
+
+.Bulk Loading Example
+====
+
+The following example shows bulk loading with Spark.
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+val hbaseContext = new HBaseContext(sc, config)
+
+val stagingFolder = ...
+val rdd = sc.parallelize(Array(
+      (Bytes.toBytes("1"),
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
+      (Bytes.toBytes("3"),
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
+
+rdd.hbaseBulkLoad(TableName.valueOf(tableName),
+  t => {
+   val rowKey = t._1
+   val family:Array[Byte] = t._2(0)._1
+   val qualifier = t._2(0)._2
+   val value = t._2(0)._3
+
+   val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
+
+   Seq((keyFamilyQualifier, value)).iterator
+  },
+  stagingFolder.getPath)
+
+val load = new LoadIncrementalHFiles(config)
+load.doBulkLoad(new Path(stagingFolder.getPath),
+  conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
+----
+====
+
+The `hbaseBulkLoad` function takes three required parameters:
+
+. The table name of the table we intend to bulk load too
+
+. A function that will convert a record in the RDD to a tuple key value par. With
+the tuple key being a KeyFamilyQualifer object and the value being the cell value.
+The KeyFamilyQualifer object will hold the RowKey, Column Family, and Column Qualifier.
+The shuffle will partition on the RowKey but will sort by all three values.
+
+. The temporary path for the HFile to be written out too
+
+Following the Spark bulk load command,  use the HBase's LoadIncrementalHFiles object
+to load the newly created HFiles into HBase.
+
+.Additional Parameters for Bulk Loading with Spark
+
+You can set the following attributes with additional parameter options on hbaseBulkLoad.
+
+* Max file size of the HFiles
+* A flag to exclude HFiles from compactions
+* Column Family settings for compression, bloomType, blockSize, and dataBlockEncoding
+
+.Using Additional Parameters
+====
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+val hbaseContext = new HBaseContext(sc, config)
+
+val stagingFolder = ...
+val rdd = sc.parallelize(Array(
+      (Bytes.toBytes("1"),
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
+      (Bytes.toBytes("3"),
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
+
+val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
+val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, "PREFIX")
+
+familyHBaseWriterOptions.put(Bytes.toBytes("columnFamily1"), f1Options)
+
+rdd.hbaseBulkLoad(TableName.valueOf(tableName),
+  t => {
+   val rowKey = t._1
+   val family:Array[Byte] = t._2(0)._1
+   val qualifier = t._2(0)._2
+   val value = t._2(0)._3
+
+   val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
+
+   Seq((keyFamilyQualifier, value)).iterator
+  },
+  stagingFolder.getPath,
+  familyHBaseWriterOptions,
+  compactionExclude = false,
+  HConstants.DEFAULT_MAX_FILE_SIZE)
+
+val load = new LoadIncrementalHFiles(config)
+load.doBulkLoad(new Path(stagingFolder.getPath),
+  conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
+----
+====
+
+Now lets look at how you would call the thin record bulk load implementation
+
+.Using thin record bulk load
+====
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+val hbaseContext = new HBaseContext(sc, config)
+
+val stagingFolder = ...
+val rdd = sc.parallelize(Array(
+      ("1",
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
+      ("3",
+        (Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
+
+rdd.hbaseBulkLoadThinRows(hbaseContext,
+      TableName.valueOf(tableName),
+      t => {
+        val rowKey = t._1
+
+        val familyQualifiersValues = new FamiliesQualifiersValues
+        t._2.foreach(f => {
+          val family:Array[Byte] = f._1
+          val qualifier = f._2
+          val value:Array[Byte] = f._3
+
+          familyQualifiersValues +=(family, qualifier, value)
+        })
+        (new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
+      },
+      stagingFolder.getPath,
+      new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
+      compactionExclude = false,
+      20)
+
+val load = new LoadIncrementalHFiles(config)
+load.doBulkLoad(new Path(stagingFolder.getPath),
+  conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
+----
+====
+
+Note that the big difference in using bulk load for thin rows is the function
+returns a tuple with the first value being the row key and the second value
+being an object of FamiliesQualifiersValues, which will contain all the
+values for this row for all column families.
+
+== SparkSQL/DataFrames
+
+HBase-Spark Connector (in HBase-Spark Module) leverages
+link:https://databricks.com/blog/2015/01/09/spark-sql-data-sources-api-unified-data-access-for-the-spark-platform.html[DataSource API]
+(link:https://issues.apache.org/jira/browse/SPARK-3247[SPARK-3247])
+introduced in Spark-1.2.0, bridges the gap between simple HBase KV store and complex
+relational SQL queries and enables users to perform complex data analytical work
+on top of HBase using Spark. HBase Dataframe is a standard Spark Dataframe, and is able to
+interact with any other data sources such as Hive, Orc, Parquet, JSON, etc.
+HBase-Spark Connector applies critical techniques such as partition pruning, column pruning,
+predicate pushdown and data locality.
+
+To use HBase-Spark connector, users need to define the Catalog for the schema mapping
+between HBase and Spark tables, prepare the data and populate the HBase table,
+then load HBase DataFrame. After that, users can do integrated query and access records
+in HBase table with SQL query. Following illustrates the basic procedure.
+
+=== Define catalog
+
+[source, scala]
+----
+def catalog = s"""{
+�������|"table":{"namespace":"default", "name":"table1"},
+�������|"rowkey":"key",
+�������|"columns":{
+���������|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
+���������|"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
+���������|"col2":{"cf":"cf2", "col":"col2", "type":"double"},
+���������|"col3":{"cf":"cf3", "col":"col3", "type":"float"},
+���������|"col4":{"cf":"cf4", "col":"col4", "type":"int"},
+���������|"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
+���������|"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
+���������|"col7":{"cf":"cf7", "col":"col7", "type":"string"},
+���������|"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
+�������|}
+�����|}""".stripMargin
+----
+
+Catalog defines a mapping between HBase and Spark tables. There are two critical parts of this catalog.
+One is the rowkey definition and the other is the mapping between table column in Spark and
+the column family and column qualifier in HBase. The above defines a schema for a HBase table
+with name as table1, row key as key and a number of columns (col1 `-` col8). Note that the rowkey
+also has to be defined in details as a column (col0), which has a specific cf (rowkey).
+
+=== Save the DataFrame
+
+[source, scala]
+----
+case class HBaseRecord(
+   col0: String,
+   col1: Boolean,
+   col2: Double,
+   col3: Float,
+   col4: Int, ������
+   col5: Long,
+   col6: Short,
+   col7: String,
+   col8: Byte)
+
+object HBaseRecord
+{ ������������������������������������������������������������������������������������������������������������
+   def apply(i: Int, t: String): HBaseRecord = {
+      val s = s"""row${"%03d".format(i)}""" ������
+      HBaseRecord(s,
+      i % 2 == 0,
+      i.toDouble,
+      i.toFloat, �
+      i,
+      i.toLong,
+      i.toShort, �
+      s"String$i: $t", �����
+      i.toByte)
+  }
+}
+
+val data = (0 to 255).map { i => �HBaseRecord(i, "extra")}
+
+sc.parallelize(data).toDF.write.options(
+�Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
+�.format("org.apache.hadoop.hbase.spark ")
+�.save()
+
+----
+`data` prepared by the user is a local Scala collection which has 256 HBaseRecord objects.
+`sc.parallelize(data)` function distributes `data` to form an RDD. `toDF` returns a DataFrame.
+`write` function returns a DataFrameWriter used to write the DataFrame to external storage
+systems (e.g. HBase here). Given a DataFrame with specified schema `catalog`, `save` function
+will create an HBase table with 5 regions and save the DataFrame inside.
+
+=== Load the DataFrame
+
+[source, scala]
+----
+def withCatalog(cat: String): DataFrame = {
+  sqlContext
+  .read
+  .options(Map(HBaseTableCatalog.tableCatalog->cat))
+  .format("org.apache.hadoop.hbase.spark")
+  .load()
+}
+val df = withCatalog(catalog)
+----
+In \u2018withCatalog\u2019 function, sqlContext is a variable of SQLContext, which is the entry point
+for working with structured data (rows and columns) in Spark.
+`read` returns a DataFrameReader that can be used to read data in as a DataFrame.
+`option` function adds input options for the underlying data source to the DataFrameReader,
+and `format` function specifies the input data source format for the DataFrameReader.
+The `load()` function loads input in as a DataFrame. The date frame `df` returned
+by `withCatalog` function could be used to access HBase table, such as 4.4 and 4.5.
+
+=== Language Integrated Query
+
+[source, scala]
+----
+val s = df.filter(($"col0" <= "row050" && $"col0" > "row040") ||
+  $"col0" === "row005" ||
+  $"col0" <= "row005")
+  .select("col0", "col1", "col4")
+s.show
+----
+DataFrame can do various operations, such as join, sort, select, filter, orderBy and so on.
+`df.filter` above filters rows using the given SQL expression. `select` selects a set of columns:
+`col0`, `col1` and `col4`.
+
+=== SQL Query
+
+[source, scala]
+----
+df.registerTempTable("table1")
+sqlContext.sql("select count(col1) from table1").show
+----
+
+`registerTempTable` registers `df` DataFrame as a temporary table using the table name `table1`.
+The lifetime of this temporary table is tied to the SQLContext that was used to create `df`.
+`sqlContext.sql` function allows the user to execute SQL queries.
+
+=== Others
+
+.Query with different timestamps
+====
+In HBaseSparkConf, four parameters related to timestamp can be set. They are TIMESTAMP,
+MIN_TIMESTAMP, MAX_TIMESTAMP and MAX_VERSIONS respectively. Users can query records with
+different timestamps or time ranges with MIN_TIMESTAMP and MAX_TIMESTAMP. In the meantime,
+use concrete value instead of tsSpecified and oldMs in the examples below.
+
+The example below shows how to load df DataFrame with different timestamps.
+tsSpecified is specified by the user.
+HBaseTableCatalog defines the HBase and Relation relation schema.
+writeCatalog defines catalog for the schema mapping.
+
+[source, scala]
+----
+val df = sqlContext.read
+      .options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMESTAMP -> tsSpecified.toString))
+      .format("org.apache.hadoop.hbase.spark")
+      .load()
+----
+
+The example below shows how to load df DataFrame with different time ranges.
+oldMs is specified by the user.
+
+[source, scala]
+----
+val df = sqlContext.read
+      .options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.MIN_TIMESTAMP -> "0",
+        HBaseSparkConf.MAX_TIMESTAMP -> oldMs.toString))
+      .format("org.apache.hadoop.hbase.spark")
+      .load()
+----
+After loading df DataFrame, users can query data.
+
+[source, scala]
+----
+df.registerTempTable("table")
+sqlContext.sql("select count(col1) from table").show
+----
+====
+
+.Native Avro support
+====
+HBase-Spark Connector support different data formats like Avro, Jason, etc. The use case below
+shows how spark supports Avro. User can persist the Avro record into HBase directly. Internally,
+the Avro schema is converted to a native Spark Catalyst data type automatically.
+Note that both key-value parts in an HBase table can be defined in Avro format.
+
+1) Define catalog for the schema mapping:
+
+[source, scala]
+----
+def catalog = s"""{
+                     |"table":{"namespace":"default", "name":"Avrotable"},
+                      |"rowkey":"key",
+                      |"columns":{
+                      |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
+                      |"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
+                      |}
+                      |}""".stripMargin
+----
+
+`catalog` is a schema for a HBase table named `Avrotable`. row key as key and
+one column col1. The rowkey also has to be defined in details as a column (col0),
+which has a specific cf (rowkey).
+
+2) Prepare the Data:
+
+[source, scala]
+----
+ object AvroHBaseRecord {
+   val schemaString =
+     s"""{"namespace": "example.avro",
+         |   "type": "record",      "name": "User",
+         |    "fields": [
+         |        {"name": "name", "type": "string"},
+         |        {"name": "favorite_number",  "type": ["int", "null"]},
+         |        {"name": "favorite_color", "type": ["string", "null"]},
+         |        {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
+         |        {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
+         |      ]    }""".stripMargin
+
+   val avroSchema: Schema = {
+     val p = new Schema.Parser
+     p.parse(schemaString)
+   }
+
+   def apply(i: Int): AvroHBaseRecord = {
+     val user = new GenericData.Record(avroSchema);
+     user.put("name", s"name${"%03d".format(i)}")
+     user.put("favorite_number", i)
+     user.put("favorite_color", s"color${"%03d".format(i)}")
+     val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
+     favoriteArray.add(s"number${i}")
+     favoriteArray.add(s"number${i+1}")
+     user.put("favorite_array", favoriteArray)
+     import collection.JavaConverters._
+     val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
+     user.put("favorite_map", favoriteMap)
+     val avroByte = AvroSedes.serialize(user, avroSchema)
+     AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
+   }
+ }
+
+ val data = (0 to 255).map { i =>
+    AvroHBaseRecord(i)
+ }
+----
+
+`schemaString` is defined first, then it is parsed to get `avroSchema`. `avroSchema` is used to
+generate `AvroHBaseRecord`. `data` prepared by users is a local Scala collection
+which has 256 `AvroHBaseRecord` objects.
+
+3) Save DataFrame:
+
+[source, scala]
+----
+ sc.parallelize(data).toDF.write.options(
+     Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
+     .format("org.apache.spark.sql.execution.datasources.hbase")
+     .save()
+----
+
+Given a data frame with specified schema `catalog`, above will create an HBase table with 5
+regions and save the data frame inside.
+
+4) Load the DataFrame
+
+[source, scala]
+----
+def avroCatalog = s"""{
+            |"table":{"namespace":"default", "name":"avrotable"},
+            |"rowkey":"key",
+            |"columns":{
+              |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
+              |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
+            |}
+          |}""".stripMargin
+
+ def withCatalog(cat: String): DataFrame = {
+     sqlContext
+         .read
+         .options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog))
+         .format("org.apache.spark.sql.execution.datasources.hbase")
+         .load()
+ }
+ val df = withCatalog(catalog)
+----
+
+In `withCatalog` function, `read` returns a DataFrameReader that can be used to read data in as a DataFrame.
+The `option` function adds input options for the underlying data source to the DataFrameReader.
+There are two options: one is to set `avroSchema` as `AvroHBaseRecord.schemaString`, and one is to
+set `HBaseTableCatalog.tableCatalog` as `avroCatalog`. The `load()` function loads input in as a DataFrame.
+The date frame `df` returned by `withCatalog` function could be used to access the HBase table.
+
+5) SQL Query
+
+[source, scala]
+----
+ df.registerTempTable("avrotable")
+ val c = sqlContext.sql("select count(1) from avrotable").
+----
+
+After loading df DataFrame, users can query data. registerTempTable registers df DataFrame
+as a temporary table using the table name avrotable. `sqlContext.sql` function allows the
+user to execute SQL queries.
+====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/thrift_filter_language.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/thrift_filter_language.adoc b/src/main/asciidoc/_chapters/thrift_filter_language.adoc
index 744cec6..da36cea 100644
--- a/src/main/asciidoc/_chapters/thrift_filter_language.adoc
+++ b/src/main/asciidoc/_chapters/thrift_filter_language.adoc
@@ -31,7 +31,6 @@
 Apache link:http://thrift.apache.org/[Thrift] is a cross-platform, cross-language development framework.
 HBase includes a Thrift API and filter language.
 The Thrift API relies on client and server processes.
-Documentation about the HBase Thrift API is located at http://wiki.apache.org/hadoop/Hbase/ThriftApi.
 
 You can configure Thrift for secure authentication at the server and client side, by following the procedures in <<security.client.thrift>> and <<security.gateway.thrift>>.
 
@@ -250,7 +249,7 @@ RowFilter::
 
 Family Filter::
   This filter takes a compare operator and a comparator.
-  It compares each qualifier name with the comparator using the compare operator and if the comparison returns true, it returns all the key-values in that column.
+  It compares each column family name with the comparator using the compare operator and if the comparison returns true, it returns all the Cells in that column family.
 
 QualifierFilter::
   This filter takes a compare operator and a comparator.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/tracing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc
index 6bb8065..0cddd8a 100644
--- a/src/main/asciidoc/_chapters/tracing.adoc
+++ b/src/main/asciidoc/_chapters/tracing.adoc
@@ -30,13 +30,13 @@
 :icons: font
 :experimental:
 
-link:https://issues.apache.org/jira/browse/HBASE-6449[HBASE-6449] added support for tracing requests through HBase, using the open source tracing library, link:http://github.com/cloudera/htrace[HTrace].
-Setting up tracing is quite simple, however it currently requires some very minor changes to your client code (it would not be very difficult to remove this requirement). 
+link:https://issues.apache.org/jira/browse/HBASE-6449[HBASE-6449] added support for tracing requests through HBase, using the open source tracing library, link:http://htrace.incubator.apache.org/[HTrace].
+Setting up tracing is quite simple, however it currently requires some very minor changes to your client code (it would not be very difficult to remove this requirement).
 
 [[tracing.spanreceivers]]
 === SpanReceivers
 
-The tracing system works by collecting information in structs called 'Spans'. It is up to you to choose how you want to receive this information by implementing the `SpanReceiver` interface, which defines one method: 
+The tracing system works by collecting information in structures called 'Spans'. It is up to you to choose how you want to receive this information by implementing the `SpanReceiver` interface, which defines one method:
 
 [source]
 ----
@@ -45,68 +45,55 @@ public void receiveSpan(Span span);
 ----
 
 This method serves as a callback whenever a span is completed.
-HTrace allows you to use as many SpanReceivers as you want so you can easily send trace information to multiple destinations. 
+HTrace allows you to use as many SpanReceivers as you want so you can easily send trace information to multiple destinations.
 
-Configure what SpanReceivers you'd like to us by putting a comma separated list of the fully-qualified class name of classes implementing `SpanReceiver` in _hbase-site.xml_ property: `hbase.trace.spanreceiver.classes`. 
+Configure what SpanReceivers you'd like to us by putting a comma separated list of the fully-qualified class name of classes implementing `SpanReceiver` in _hbase-site.xml_ property: `hbase.trace.spanreceiver.classes`.
 
 HTrace includes a `LocalFileSpanReceiver` that writes all span information to local files in a JSON-based format.
-The `LocalFileSpanReceiver` looks in _hbase-site.xml_      for a `hbase.local-file-span-receiver.path` property with a value describing the name of the file to which nodes should write their span information. 
+The `LocalFileSpanReceiver` looks in _hbase-site.xml_      for a `hbase.local-file-span-receiver.path` property with a value describing the name of the file to which nodes should write their span information.
 
 [source]
 ----
 
 <property>
   <name>hbase.trace.spanreceiver.classes</name>
-  <value>org.htrace.impl.LocalFileSpanReceiver</value>
+  <value>org.apache.htrace.impl.LocalFileSpanReceiver</value>
 </property>
 <property>
-  <name>hbase.local-file-span-receiver.path</name>
+  <name>hbase.htrace.local-file-span-receiver.path</name>
   <value>/var/log/hbase/htrace.out</value>
 </property>
 ----
 
-HTrace also provides `ZipkinSpanReceiver` which converts spans to link:http://github.com/twitter/zipkin[Zipkin] span format and send them to Zipkin server.
-In order to use this span receiver, you need to install the jar of htrace-zipkin to your HBase's classpath on all of the nodes in your cluster. 
+HTrace also provides `ZipkinSpanReceiver` which converts spans to link:http://github.com/twitter/zipkin[Zipkin] span format and send them to Zipkin server. In order to use this span receiver, you need to install the jar of htrace-zipkin to your HBase's classpath on all of the nodes in your cluster.
 
-_htrace-zipkin_ is published to the maven central repository.
-You could get the latest version from there or just build it locally and then copy it out to all nodes, change your config to use zipkin receiver, distribute the new configuration and then (rolling) restart. 
+_htrace-zipkin_ is published to the link:http://search.maven.org/#search%7Cgav%7C1%7Cg%3A%22org.apache.htrace%22%20AND%20a%3A%22htrace-zipkin%22[Maven central repository]. You could get the latest version from there or just build it locally (see the link:http://htrace.incubator.apache.org/[HTrace] homepage for information on how to do this) and then copy it out to all nodes.
 
-Here is the example of manual setup procedure. 
-
-----
-
-$ git clone https://github.com/cloudera/htrace
-$ cd htrace/htrace-zipkin
-$ mvn compile assembly:single
-$ cp target/htrace-zipkin-*-jar-with-dependencies.jar $HBASE_HOME/lib/
-  # copy jar to all nodes...
-----
-
-The `ZipkinSpanReceiver` looks in _hbase-site.xml_      for a `hbase.zipkin.collector-hostname` and `hbase.zipkin.collector-port` property with a value describing the Zipkin collector server to which span information are sent. 
+`ZipkinSpanReceiver` for properties called `hbase.htrace.zipkin.collector-hostname` and `hbase.htrace.zipkin.collector-port` in _hbase-site.xml_ with values describing the Zipkin collector server to which span information are sent.
 
 [source,xml]
 ----
 
 <property>
   <name>hbase.trace.spanreceiver.classes</name>
-  <value>org.htrace.impl.ZipkinSpanReceiver</value>
-</property> 
+  <value>org.apache.htrace.impl.ZipkinSpanReceiver</value>
+</property>
 <property>
-  <name>hbase.zipkin.collector-hostname</name>
+  <name>hbase.htrace.zipkin.collector-hostname</name>
   <value>localhost</value>
-</property> 
+</property>
 <property>
-  <name>hbase.zipkin.collector-port</name>
+  <name>hbase.htrace.zipkin.collector-port</name>
   <value>9410</value>
 </property>
 ----
 
-If you do not want to use the included span receivers, you are encouraged to write your own receiver (take a look at `LocalFileSpanReceiver` for an example). If you think others would benefit from your receiver, file a JIRA or send a pull request to link:http://github.com/cloudera/htrace[HTrace]. 
+If you do not want to use the included span receivers, you are encouraged to write your own receiver (take a look at `LocalFileSpanReceiver` for an example). If you think others would benefit from your receiver, file a JIRA with the HTrace project.
 
 [[tracing.client.modifications]]
 == Client Modifications
 
-In order to turn on tracing in your client code, you must initialize the module sending spans to receiver once per client process. 
+In order to turn on tracing in your client code, you must initialize the module sending spans to receiver once per client process.
 
 [source,java]
 ----
@@ -120,7 +107,7 @@ private SpanReceiverHost spanReceiverHost;
 ----
 
 Then you simply start tracing span before requests you think are interesting, and close it when the request is done.
-For example, if you wanted to trace all of your get operations, you change this: 
+For example, if you wanted to trace all of your get operations, you change this:
 
 [source,java]
 ----
@@ -131,7 +118,7 @@ Get get = new Get(Bytes.toBytes("r1"));
 Result res = table.get(get);
 ----
 
-into: 
+into:
 
 [source,java]
 ----
@@ -146,7 +133,7 @@ try {
 }
 ----
 
-If you wanted to trace half of your 'get' operations, you would pass in: 
+If you wanted to trace half of your 'get' operations, you would pass in:
 
 [source,java]
 ----
@@ -155,13 +142,12 @@ new ProbabilitySampler(0.5)
 ----
 
 in lieu of `Sampler.ALWAYS` to `Trace.startSpan()`.
-See the HTrace _README_ for more information on Samplers. 
+See the HTrace _README_ for more information on Samplers.
 
 [[tracing.client.shell]]
 == Tracing from HBase Shell
 
-You can use +trace+ command for tracing requests from HBase Shell. +trace 'start'+ command turns on tracing and +trace
-        'stop'+ command turns off tracing. 
+You can use `trace` command for tracing requests from HBase Shell. `trace 'start'` command turns on tracing and `trace 'stop'` command turns off tracing.
 
 [source]
 ----
@@ -171,9 +157,8 @@ hbase(main):002:0> put 'test', 'row1', 'f:', 'val1'   # traced commands
 hbase(main):003:0> trace 'stop'
 ----
 
-+trace 'start'+ and +trace 'stop'+ always returns boolean value representing if or not there is ongoing tracing.
-As a result, +trace
-        'stop'+ returns false on suceess. +trace 'status'+ just returns if or not tracing is turned on. 
+`trace 'start'` and `trace 'stop'` always returns boolean value representing if or not there is ongoing tracing.
+As a result, `trace 'stop'` returns false on success. `trace 'status'` just returns if or not tracing is turned on.
 
 [source]
 ----

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/troubleshooting.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/troubleshooting.adoc b/src/main/asciidoc/_chapters/troubleshooting.adoc
index 1776c9e..ce47423 100644
--- a/src/main/asciidoc/_chapters/troubleshooting.adoc
+++ b/src/main/asciidoc/_chapters/troubleshooting.adoc
@@ -41,7 +41,7 @@ RegionServer suicides are 'normal', as this is what they do when something goes
 For example, if ulimit and max transfer threads (the two most important initial settings, see <<ulimit>> and <<dfs.datanode.max.transfer.threads>>) aren't changed, it will make it impossible at some point for DataNodes to create new threads that from the HBase point of view is seen as if HDFS was gone.
 Think about what would happen if your MySQL database was suddenly unable to access files on your local file system, well it's the same with HBase and HDFS.
 Another very common reason to see RegionServers committing seppuku is when they enter prolonged garbage collection pauses that last longer than the default ZooKeeper session timeout.
-For more information on GC pauses, see the link:http://www.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
+For more information on GC pauses, see the link:https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
 
 [[trouble.log]]
 == Logs
@@ -89,11 +89,11 @@ Additionally, each DataNode server will also have a TaskTracker/NodeManager log
 [[rpc.logging]]
 ==== Enabling RPC-level logging
 
-Enabling the RPC-level logging on a RegionServer can often given insight on timings at the server.
+Enabling the RPC-level logging on a RegionServer can often give insight on timings at the server.
 Once enabled, the amount of log spewed is voluminous.
 It is not recommended that you leave this logging on for more than short bursts of time.
 To enable RPC-level logging, browse to the RegionServer UI and click on _Log Level_.
-Set the log level to `DEBUG` for the package `org.apache.hadoop.ipc` (Thats right, for `hadoop.ipc`, NOT, `hbase.ipc`). Then tail the RegionServers log.
+Set the log level to `DEBUG` for the package `org.apache.hadoop.ipc` (That's right, for `hadoop.ipc`, NOT, `hbase.ipc`). Then tail the RegionServers log.
 Analyze.
 
 To disable, set the logging level back to `INFO` level.
@@ -185,7 +185,7 @@ The key points here is to keep all these pauses low.
 CMS pauses are always low, but if your ParNew starts growing, you can see minor GC pauses approach 100ms, exceed 100ms and hit as high at 400ms.
 
 This can be due to the size of the ParNew, which should be relatively small.
-If your ParNew is very large after running HBase for a while, in one example a ParNew was about 150MB, then you might have to constrain the size of ParNew (The larger it is, the longer the collections take but if its too small, objects are promoted to old gen too quickly). In the below we constrain new gen size to 64m.
+If your ParNew is very large after running HBase for a while, in one example a ParNew was about 150MB, then you might have to constrain the size of ParNew (The larger it is, the longer the collections take but if it's too small, objects are promoted to old gen too quickly). In the below we constrain new gen size to 64m.
 
 Add the below line in _hbase-env.sh_:
 [source,bourne]
@@ -211,7 +211,7 @@ Similarly, to enable GC logging for client processes, uncomment one of the below
 # If <FILE-PATH> is not replaced, the log file(.gc) would be generated in the HBASE_LOG_DIR .
 ----
 
-For more information on GC pauses, see the link:http://www.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
+For more information on GC pauses, see the link:https://blog.cloudera.com/blog/2011/02/avoiding-full-gcs-in-hbase-with-memstore-local-allocation-buffers-part-1/[3 part blog post] by Todd Lipcon and <<gcpause>> above.
 
 [[trouble.resources]]
 == Resources
@@ -443,7 +443,7 @@ java.lang.Thread.State: WAITING (on object monitor)
     at org.apache.hadoop.hbase.regionserver.MemStoreFlusher.run(MemStoreFlusher.java:146)
 ----
 
-A handler thread that's waiting for stuff to do (like put, delete, scan, etc):
+A handler thread that's waiting for stuff to do (like put, delete, scan, etc.):
 
 [source]
 ----
@@ -557,7 +557,15 @@ You can also tail all the logs at the same time, edit files, etc.
 [[trouble.client]]
 == Client
 
-For more information on the HBase client, see <<client,client>>.
+For more information on the HBase client, see <<architecture.client,client>>.
+
+=== Missed Scan Results Due To Mismatch Of `hbase.client.scanner.max.result.size` Between Client and Server
+If either the client or server version is lower than 0.98.11/1.0.0 and the server
+has a smaller value for `hbase.client.scanner.max.result.size` than the client, scan
+requests that reach the server's `hbase.client.scanner.max.result.size` are likely
+to miss data. In particular, 0.98.11 defaults `hbase.client.scanner.max.result.size`
+to 2 MB but other versions default to larger values. For this reason, be very careful
+using 0.98.11 servers with any other client version.
 
 [[trouble.client.scantimeout]]
 === ScannerTimeoutException or UnknownScannerException
@@ -834,6 +842,34 @@ Two common use-cases for querying HDFS for HBase objects is research the degree
 If there are a large number of StoreFiles for each ColumnFamily it could indicate the need for a major compaction.
 Additionally, after a major compaction if the resulting StoreFile is "small" it could indicate the need for a reduction of ColumnFamilies for the table.
 
+=== Unexpected Filesystem Growth
+
+If you see an unexpected spike in filesystem usage by HBase, two possible culprits
+are snapshots and WALs.
+
+Snapshots::
+  When you create a snapshot, HBase retains everything it needs to recreate the table's
+  state at that time of the snapshot. This includes deleted cells or expired versions.
+  For this reason, your snapshot usage pattern should be well-planned, and you should
+  prune snapshots that you no longer need. Snapshots are stored in `/hbase/.snapshots`,
+  and archives needed to restore snapshots are stored in
+  `/hbase/.archive/<_tablename_>/<_region_>/<_column_family_>/`.
+
+  *Do not* manage snapshots or archives manually via HDFS. HBase provides APIs and
+  HBase Shell commands for managing them. For more information, see <<ops.snapshots>>.
+
+WAL::
+  Write-ahead logs (WALs) are stored in subdirectories of the HBase root directory,
+  typically `/hbase/`, depending on their status. Already-processed WALs are stored
+  in `/hbase/oldWALs/` and corrupt WALs are stored in `/hbase/.corrupt/` for examination.
+  If the size of one of these subdirectories is growing, examine the HBase
+  server logs to find the root cause for why WALs are not being processed correctly.
++
+If you use replication and `/hbase/oldWALs/` is using more space than you expect,
+remember that WALs are saved when replication is disabled, as long as there are peers.
+
+*Do not* manage WALs manually via HDFS.
+
 [[trouble.network]]
 == Network
 
@@ -1037,7 +1073,7 @@ However, if the NotServingRegionException is logged ERROR, then the client ran o
 
 Fix your DNS.
 In versions of Apache HBase before 0.92.x, reverse DNS needs to give same answer as forward lookup.
-See link:https://issues.apache.org/jira/browse/HBASE-3431[HBASE 3431 RegionServer is not using the name given it by the master; double entry in master listing of servers] for gorey details.
+See link:https://issues.apache.org/jira/browse/HBASE-3431[HBASE 3431 RegionServer is not using the name given it by the master; double entry in master listing of servers] for gory details.
 
 [[brand.new.compressor]]
 ==== Logs flooded with '2011-01-10 12:40:48,407 INFO org.apache.hadoop.io.compress.CodecPool: Gotbrand-new compressor' messages
@@ -1082,7 +1118,7 @@ to use. Was=myhost-1234, Now=ip-10-55-88-99.ec2.internal
 [[trouble.master]]
 == Master
 
-For more information on the Master, see <<master,master>>.
+For more information on the Master, see <<architecture.master,master>>.
 
 [[trouble.master.startup]]
 === Startup Errors
@@ -1314,6 +1350,28 @@ Settings for HDFS retries and timeouts are important to HBase.::
   Defaults are current as of Hadoop 2.3.
   Check the Hadoop documentation for the most current values and recommendations.
 
+The HBase Balancer and HDFS Balancer are incompatible::
+  The HDFS balancer attempts to spread HDFS blocks evenly among DataNodes. HBase relies
+  on compactions to restore locality after a region split or failure. These two types
+  of balancing do not work well together.
++
+In the past, the generally accepted advice was to turn off the HDFS load balancer and rely
+on the HBase balancer, since the HDFS balancer would degrade locality. This advice
+is still valid if your HDFS version is lower than 2.7.1.
++
+link:https://issues.apache.org/jira/browse/HDFS-6133[HDFS-6133] provides the ability
+to exclude a given directory from the HDFS load balancer, by setting the
+`dfs.datanode.block-pinning.enabled` property to `true` in your HDFS
+configuration and running the following hdfs command:
++
+----
+$ sudo -u hdfs hdfs balancer -exclude /hbase
+----
++
+NOTE: HDFS-6133 is available in HDFS 2.7.0 and higher, but HBase does not support
+running on HDFS 2.7.0, so you must be using HDFS 2.7.1 or higher to use this feature
+with HBase.
+
 .Connection Timeouts
 Connection timeouts occur between the client (HBASE) and the HDFS DataNode.
 They may occur when establishing a connection, attempting to read, or attempting to write.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/unit_testing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/unit_testing.adoc b/src/main/asciidoc/_chapters/unit_testing.adoc
index 3f70001..0c4d812 100644
--- a/src/main/asciidoc/_chapters/unit_testing.adoc
+++ b/src/main/asciidoc/_chapters/unit_testing.adoc
@@ -47,7 +47,7 @@ public class MyHBaseDAO {
         Put put = createPut(obj);
         table.put(put);
     }
-    
+
     private static Put createPut(HBaseTestObj obj) {
         Put put = new Put(Bytes.toBytes(obj.getRowKey()));
         put.add(Bytes.toBytes("CF"), Bytes.toBytes("CQ-1"),
@@ -96,13 +96,14 @@ public class TestMyHbaseDAOData {
 
 These tests ensure that your `createPut` method creates, populates, and returns a `Put` object with expected values.
 Of course, JUnit can do much more than this.
-For an introduction to JUnit, see link:https://github.com/junit-team/junit/wiki/Getting-started. 
+For an introduction to JUnit, see https://github.com/junit-team/junit/wiki/Getting-started.
 
+[[mockito]]
 == Mockito
 
 Mockito is a mocking framework.
 It goes further than JUnit by allowing you to test the interactions between objects without having to replicate the entire environment.
-You can read more about Mockito at its project site, link:https://code.google.com/p/mockito/.
+You can read more about Mockito at its project site, https://code.google.com/p/mockito/.
 
 You can use Mockito to do unit testing on smaller units.
 For instance, you can mock a `org.apache.hadoop.hbase.Server` instance or a `org.apache.hadoop.hbase.master.MasterServices` interface reference rather than a full-blown `org.apache.hadoop.hbase.master.HMaster`.
@@ -133,7 +134,7 @@ public class TestMyHBaseDAO{
   Configuration config = HBaseConfiguration.create();
   @Mock
   Connection connection = ConnectionFactory.createConnection(config);
-  @Mock 
+  @Mock
   private Table table;
   @Captor
   private ArgumentCaptor putCaptor;
@@ -150,7 +151,7 @@ public class TestMyHBaseDAO{
     MyHBaseDAO.insertRecord(table, obj);
     verify(table).put(putCaptor.capture());
     Put put = putCaptor.getValue();
-  
+
     assertEquals(Bytes.toString(put.getRow()), obj.getRowKey());
     assert(put.has(Bytes.toBytes("CF"), Bytes.toBytes("CQ-1")));
     assert(put.has(Bytes.toBytes("CF"), Bytes.toBytes("CQ-2")));
@@ -182,7 +183,7 @@ public class MyReducer extends TableReducer<Text, Text, ImmutableBytesWritable>
    public static final byte[] CF = "CF".getBytes();
    public static final byte[] QUALIFIER = "CQ-1".getBytes();
    public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
-     //bunch of processing to extract data to be inserted, in our case, lets say we are simply
+     //bunch of processing to extract data to be inserted, in our case, let's say we are simply
      //appending all the records we receive from the mapper for this particular
      //key and insert one record into HBase
      StringBuffer data = new StringBuffer();
@@ -197,7 +198,7 @@ public class MyReducer extends TableReducer<Text, Text, ImmutableBytesWritable>
  }
 ----
 
-To test this code, the first step is to add a dependency to MRUnit to your Maven POM file. 
+To test this code, the first step is to add a dependency to MRUnit to your Maven POM file.
 
 [source,xml]
 ----
@@ -225,16 +226,16 @@ public class MyReducerTest {
       MyReducer reducer = new MyReducer();
       reduceDriver = ReduceDriver.newReduceDriver(reducer);
     }
-  
+
    @Test
    public void testHBaseInsert() throws IOException {
-      String strKey = "RowKey-1", strValue = "DATA", strValue1 = "DATA1", 
+      String strKey = "RowKey-1", strValue = "DATA", strValue1 = "DATA1",
 strValue2 = "DATA2";
       List<Text> list = new ArrayList<Text>();
       list.add(new Text(strValue));
       list.add(new Text(strValue1));
       list.add(new Text(strValue2));
-      //since in our case all that the reducer is doing is appending the records that the mapper   
+      //since in our case all that the reducer is doing is appending the records that the mapper
       //sends it, we should get the following back
       String expectedOutput = strValue + strValue1 + strValue2;
      //Setup Input, mimic what mapper would have passed
@@ -242,10 +243,10 @@ strValue2 = "DATA2";
       reduceDriver.withInput(new Text(strKey), list);
       //run the reducer and get its output
       List<Pair<ImmutableBytesWritable, Writable>> result = reduceDriver.run();
-    
+
       //extract key from result and verify
       assertEquals(Bytes.toString(result.get(0).getFirst().get()), strKey);
-    
+
       //extract value for CF/QUALIFIER and verify
       Put a = (Put)result.get(0).getSecond();
       String c = Bytes.toString(a.get(CF, QUALIFIER).get(0).getValue());
@@ -259,7 +260,7 @@ Your MRUnit test verifies that the output is as expected, the Put that is insert
 
 MRUnit includes a MapperDriver to test mapping jobs, and you can use MRUnit to test other operations, including reading from HBase, processing data, or writing to HDFS,
 
-== Integration Testing with a HBase Mini-Cluster
+== Integration Testing with an HBase Mini-Cluster
 
 HBase ships with HBaseTestingUtility, which makes it easy to write integration tests using a [firstterm]_mini-cluster_.
 The first step is to add some dependencies to your Maven POM file.
@@ -267,37 +268,18 @@ Check the versions to be sure they are appropriate.
 
 [source,xml]
 ----
+<properties>
+  <hbase.version>2.0.0-SNAPSHOT</hbase.version>
+</properties>
 
-<dependency>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-common</artifactId>
-    <version>2.0.0</version>
-    <type>test-jar</type>
-    <scope>test</scope>
-</dependency>
-
-<dependency>
+<dependencies>
+  <dependency>
     <groupId>org.apache.hbase</groupId>
-    <artifactId>hbase</artifactId>
-    <version>0.98.3</version>
-    <type>test-jar</type>
+    <artifactId>hbase-testing-util</artifactId>
+    <version>${hbase.version}</version>
     <scope>test</scope>
-</dependency>
-        
-<dependency>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-hdfs</artifactId>
-    <version>2.0.0</version>
-    <type>test-jar</type>
-    <scope>test</scope>
-</dependency>
-
-<dependency>
-    <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-hdfs</artifactId>
-    <version>2.0.0</version>
-    <scope>test</scope>
-</dependency>
+  </dependency>
+</dependencies>
 ----
 
 This code represents an integration test for the MyDAO insert shown in <<unit.tests,unit.tests>>.
@@ -308,8 +290,9 @@ This code represents an integration test for the MyDAO insert shown in <<unit.te
 public class MyHBaseIntegrationTest {
     private static HBaseTestingUtility utility;
     byte[] CF = "CF".getBytes();
-    byte[] QUALIFIER = "CQ-1".getBytes();
-    
+    byte[] CQ1 = "CQ-1".getBytes();
+    byte[] CQ2 = "CQ-2".getBytes();
+
     @Before
     public void setup() throws Exception {
     	utility = new HBaseTestingUtility();
@@ -318,8 +301,7 @@ public class MyHBaseIntegrationTest {
 
     @Test
         public void testInsert() throws Exception {
-       	 HTableInterface table = utility.createTable(Bytes.toBytes("MyTest"),
-       			 Bytes.toBytes("CF"));
+       	 HTableInterface table = utility.createTable(Bytes.toBytes("MyTest"), CF);
        	 HBaseTestObj obj = new HBaseTestObj();
        	 obj.setRowKey("ROWKEY-1");
        	 obj.setData1("DATA-1");
@@ -343,7 +325,7 @@ This code creates an HBase mini-cluster and starts it.
 Next, it creates a table called `MyTest` with one column family, `CF`.
 A record is inserted, a Get is performed from the same table, and the insertion is verified.
 
-NOTE: Starting the mini-cluster takes about 20-30 seconds, but that should be appropriate for integration testing. 
+NOTE: Starting the mini-cluster takes about 20-30 seconds, but that should be appropriate for integration testing.
 
 To use an HBase mini-cluster on Microsoft Windows, you need to use a Cygwin environment.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index 6b63833..9552024 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -31,7 +31,7 @@ You cannot skip major versions when upgrading. If you are upgrading from version
 
 NOTE: It may be possible to skip across versions -- for example go from 0.92.2 straight to 0.98.0 just following the 0.96.x upgrade instructions -- but these scenarios are untested.
 
-Review <<configuration>>, in particular <<hadoop>>.
+Review <<configuration>>, in particular <<hadoop>>. Familiarize yourself with <<hbase_supported_tested_definitions>>.
 
 [[hbase.versioning]]
 == HBase version number and compatibility
@@ -71,9 +71,10 @@ In addition to the usual API versioning considerations HBase has other compatibi
 
 .Client API compatibility
 * Allow changing or removing existing client APIs.
-* An API needs to deprecated for a major version before we will change/remove it.
+* An API needs to be deprecated for a major version before we will change/remove it.
 * APIs available in a patch version will be available in all later patch versions. However, new APIs may be added which will not be available in earlier patch versions.
-* Example: A user using a newly deprecated api does not need to modify application code with hbase api calls until the next major version.
+* New APIs introduced in a patch version will only be added in a source compatible way footnote:[See 'Source Compatibility' https://blogs.oracle.com/darcy/entry/kinds_of_compatibility]: i.e. code that implements public APIs will continue to compile.
+* Example: A user using a newly deprecated API does not need to modify application code with HBase API calls until the next major version.
 
 .Client Binary compatibility
 * Client code written to APIs available in a given patch release can run unchanged (no recompilation needed) against the new jars of later patch versions.
@@ -92,10 +93,10 @@ In addition to the usual API versioning considerations HBase has other compatibi
 .Operational Compatibility
 * Metric changes
 * Behavioral changes of services
-* Web page APIs
+* JMX APIs exposed via the `/jmx/` endpoint
 
 .Summary
-* A patch upgrade is a drop-in replacement. Any change that is not Java binary compatible would not be allowed.footnote:[See http://docs.oracle.com/javase/specs/jls/se7/html/jls-13.html.]. Downgrading versions within patch releases may not be compatible.
+* A patch upgrade is a drop-in replacement. Any change that is not Java binary and source compatible would not be allowed.footnote:[See http://docs.oracle.com/javase/specs/jls/se7/html/jls-13.html.] Downgrading versions within patch releases may not be compatible.
 
 * A minor upgrade requires no application/client code modification. Ideally it would be a drop-in replacement but client code, coprocessors, filters, etc might have to be recompiled if new jars are used.
 
@@ -132,7 +133,7 @@ HBase Client API::
 
 [[hbase.limitetprivate.api]]
 HBase LimitedPrivate API::
-  LimitedPrivate annotation comes with a set of target consumers for the interfaces. Those consumers are coprocessors, phoenix, replication endpoint implemnetations or similar. At this point, HBase only guarantees source and binary compatibility for these interfaces between patch versions.
+  LimitedPrivate annotation comes with a set of target consumers for the interfaces. Those consumers are coprocessors, phoenix, replication endpoint implementations or similar. At this point, HBase only guarantees source and binary compatibility for these interfaces between patch versions.
 
 [[hbase.private.api]]
 HBase Private API::
@@ -158,7 +159,7 @@ When we say two HBase versions are compatible, we mean that the versions are wir
 
 A rolling upgrade is the process by which you update the servers in your cluster a server at a time. You can rolling upgrade across HBase versions if they are binary or wire compatible. See <<hbase.rolling.restart>> for more on what this means. Coarsely, a rolling upgrade is a graceful stop each server, update the software, and then restart. You do this for each server in the cluster. Usually you upgrade the Master first and then the RegionServers. See <<rolling>> for tools that can help use the rolling upgrade process.
 
-For example, in the below, HBase was symlinked to the actual HBase install. On upgrade, before running a rolling restart over the cluser, we changed the symlink to point at the new HBase software version and then ran
+For example, in the below, HBase was symlinked to the actual HBase install. On upgrade, before running a rolling restart over the cluster, we changed the symlink to point at the new HBase software version and then ran
 
 [source,bash]
 ----
@@ -192,9 +193,15 @@ See <<zookeeper.requirements>>.
 .HBase Default Ports Changed
 The ports used by HBase changed. They used to be in the 600XX range. In HBase 1.0.0 they have been moved up out of the ephemeral port range and are 160XX instead (Master web UI was 60010 and is now 16010; the RegionServer web UI was 60030 and is now 16030, etc.). If you want to keep the old port locations, copy the port setting configs from _hbase-default.xml_ into _hbase-site.xml_, change them back to the old values from the HBase 0.98.x era, and ensure you've distributed your configurations before you restart.
 
+.HBase Master Port Binding Change
+In HBase 1.0.x, the HBase Master binds the RegionServer ports as well as the Master
+ports. This behavior is changed from HBase versions prior to 1.0. In HBase 1.1 and 2.0 branches,
+this behavior is reverted to the pre-1.0 behavior of the HBase master not binding the RegionServer
+ports.
+
 [[upgrade1.0.hbase.bucketcache.percentage.in.combinedcache]]
 .hbase.bucketcache.percentage.in.combinedcache configuration has been REMOVED
-You may have made use of this configuration if you are using BucketCache. If NOT using BucketCache, this change does not effect you. Its removal means that your L1 LruBlockCache is now sized using `hfile.block.cache.size` -- i.e. the way you would size the on-heap L1 LruBlockCache if you were NOT doing BucketCache -- and the BucketCache size is not whatever the setting for `hbase.bucketcache.size` is. You may need to adjust configs to get the LruBlockCache and BucketCache sizes set to what they were in 0.98.x and previous. If you did not set this config., its default value was 0.9. If you do nothing, your BucketCache will increase in size by 10%. Your L1 LruBlockCache will become `hfile.block.cache.size` times your java heap size (`hfile.block.cache.size` is a float between 0.0 and 1.0). To read more, see link:https://issues.apache.org/jira/browse/HBASE-11520[HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"].
+You may have made use of this configuration if you are using BucketCache. If NOT using BucketCache, this change does not affect you. Its removal means that your L1 LruBlockCache is now sized using `hfile.block.cache.size` -- i.e. the way you would size the on-heap L1 LruBlockCache if you were NOT doing BucketCache -- and the BucketCache size is not whatever the setting for `hbase.bucketcache.size` is. You may need to adjust configs to get the LruBlockCache and BucketCache sizes set to what they were in 0.98.x and previous. If you did not set this config., its default value was 0.9. If you do nothing, your BucketCache will increase in size by 10%. Your L1 LruBlockCache will become `hfile.block.cache.size` times your java heap size (`hfile.block.cache.size` is a float between 0.0 and 1.0). To read more, see link:https://issues.apache.org/jira/browse/HBASE-11520[HBASE-11520 Simplify offheap cache config by removing the confusing "hbase.bucketcache.percentage.in.combinedcache"].
 
 [[hbase-12068]]
 .If you have your own customer filters.
@@ -204,6 +211,14 @@ See the release notes on the issue link:https://issues.apache.org/jira/browse/HB
 .Distributed Log Replay
 <<distributed.log.replay>> is off by default in HBase 1.0.0. Enabling it can make a big difference improving HBase MTTR. Enable this feature if you are doing a clean stop/start when you are upgrading. You cannot rolling upgrade to this feature (caveat if you are running on a version of HBase in excess of HBase 0.98.4 -- see link:https://issues.apache.org/jira/browse/HBASE-12577[HBASE-12577 Disable distributed log replay by default] for more).
 
+.Mismatch Of `hbase.client.scanner.max.result.size` Between Client and Server
+If either the client or server version is lower than 0.98.11/1.0.0 and the server
+has a smaller value for `hbase.client.scanner.max.result.size` than the client, scan
+requests that reach the server's `hbase.client.scanner.max.result.size` are likely
+to miss data. In particular, 0.98.11 defaults `hbase.client.scanner.max.result.size`
+to 2 MB but other versions default to larger values. For this reason, be very careful
+using 0.98.11 servers with any other client version.
+
 [[upgrade1.0.rolling.upgrade]]
 ==== Rolling upgrade from 0.98.x to HBase 1.0.0
 .From 0.96.x to 1.0.0
@@ -378,7 +393,7 @@ The migration is a one-time event. However, every time your cluster starts, `MET
 
 [[upgrade0.94]]
 === Upgrading from 0.92.x to 0.94.x
-We used to think that 0.92 and 0.94 were interface compatible and that you can do a rolling upgrade between these versions but then we figured that link:https://issues.apache.org/jira/browse/HBASE-5357[HBASE-5357 Use builder pattern in HColumnDescriptor] changed method signatures so rather than return `void` they instead return `HColumnDescriptor`. This will throw`java.lang.NoSuchMethodError: org.apache.hadoop.hbase.HColumnDescriptor.setMaxVersions(I)V` so 0.92 and 0.94 are NOT compatible. You cannot do a rolling upgrade between them.
+We used to think that 0.92 and 0.94 were interface compatible and that you can do a rolling upgrade between these versions but then we figured that link:https://issues.apache.org/jira/browse/HBASE-5357[HBASE-5357 Use builder pattern in HColumnDescriptor] changed method signatures so rather than return `void` they instead return `HColumnDescriptor`. This will throw `java.lang.NoSuchMethodError: org.apache.hadoop.hbase.HColumnDescriptor.setMaxVersions(I)V` so 0.92 and 0.94 are NOT compatible. You cannot do a rolling upgrade between them.
 
 [[upgrade0.92]]
 === Upgrading from 0.90.x to 0.92.x

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/ycsb.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ycsb.adoc b/src/main/asciidoc/_chapters/ycsb.adoc
index d8ec628..f843756 100644
--- a/src/main/asciidoc/_chapters/ycsb.adoc
+++ b/src/main/asciidoc/_chapters/ycsb.adoc
@@ -20,6 +20,7 @@
 ////
 
 [appendix]
+[[ycsb]]
 == YCSB
 :doctype: book
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/zookeeper.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/zookeeper.adoc b/src/main/asciidoc/_chapters/zookeeper.adoc
index f6134b7..91577da 100644
--- a/src/main/asciidoc/_chapters/zookeeper.adoc
+++ b/src/main/asciidoc/_chapters/zookeeper.adoc
@@ -35,7 +35,7 @@ You can also manage the ZooKeeper ensemble independent of HBase and just point H
 To toggle HBase management of ZooKeeper, use the `HBASE_MANAGES_ZK` variable in _conf/hbase-env.sh_.
 This variable, which defaults to `true`, tells HBase whether to start/stop the ZooKeeper ensemble servers as part of HBase start/stop.
 
-When HBase manages the ZooKeeper ensemble, you can specify ZooKeeper configuration using its native _zoo.cfg_ file, or, the easier option is to just specify ZooKeeper options directly in _conf/hbase-site.xml_.
+When HBase manages the ZooKeeper ensemble, you can specify ZooKeeper configuration directly in _conf/hbase-site.xml_.
 A ZooKeeper configuration option can be set as a property in the HBase _hbase-site.xml_ XML configuration file by prefacing the ZooKeeper option name with `hbase.zookeeper.property`.
 For example, the `clientPort` setting in ZooKeeper can be changed by setting the `hbase.zookeeper.property.clientPort` property.
 For all default values used by HBase, including ZooKeeper configuration, see <<hbase_default_configurations,hbase default configurations>>.
@@ -45,7 +45,7 @@ HBase does not ship with a _zoo.cfg_ so you will need to browse the _conf_ direc
 
 You must at least list the ensemble servers in _hbase-site.xml_ using the `hbase.zookeeper.quorum` property.
 This property defaults to a single ensemble member at `localhost` which is not suitable for a fully distributed HBase.
-(It binds to the local machine only and remote clients will not be able to connect). 
+(It binds to the local machine only and remote clients will not be able to connect).
 
 .How many ZooKeepers should I run?
 [NOTE]
@@ -54,7 +54,7 @@ You can run a ZooKeeper ensemble that comprises 1 node only but in production it
 Also, run an odd number of machines.
 In ZooKeeper, an even number of peers is supported, but it is normally not used because an even sized ensemble requires, proportionally, more peers to form a quorum than an odd sized ensemble requires.
 For example, an ensemble with 4 peers requires 3 to form a quorum, while an ensemble with 5 also requires 3 to form a quorum.
-Thus, an ensemble of 5 allows 2 peers to fail, and thus is more fault tolerant than the ensemble of 4, which allows only 1 down peer. 
+Thus, an ensemble of 5 allows 2 peers to fail, and thus is more fault tolerant than the ensemble of 4, which allows only 1 down peer.
 
 Give each ZooKeeper server around 1GB of RAM, and if possible, its own dedicated disk (A dedicated disk is the best thing you can do to ensure a performant ZooKeeper ensemble). For very heavily loaded clusters, run ZooKeeper servers on separate machines from RegionServers (DataNodes and TaskTrackers).
 ====
@@ -97,18 +97,16 @@ In the example below we have ZooKeeper persist to _/user/local/zookeeper_.
   </configuration>
 ----
 
-.What verion of ZooKeeper should I use?
+.What version of ZooKeeper should I use?
 [CAUTION]
 ====
-The newer version, the better.
-For example, some folks have been bitten by link:https://issues.apache.org/jira/browse/ZOOKEEPER-1277[ZOOKEEPER-1277].
-If running zookeeper 3.5+, you can ask hbase to make use of the new multi operation by enabling <<hbase.zookeeper.usemulti,hbase.zookeeper.useMulti>>" in your _hbase-site.xml_. 
+The newer version, the better. ZooKeeper 3.4.x is required as of HBase 1.0.0
 ====
 
 .ZooKeeper Maintenance
 [CAUTION]
 ====
-Be sure to set up the data dir cleaner described under link:http://zookeeper.apache.org/doc/r3.1.2/zookeeperAdmin.html#sc_maintenance[Zookeeper
+Be sure to set up the data dir cleaner described under link:http://zookeeper.apache.org/doc/r3.1.2/zookeeperAdmin.html#sc_maintenance[ZooKeeper
         Maintenance] else you could have 'interesting' problems a couple of months in; i.e.
 zookeeper could start dropping sessions if it has to run through a directory of hundreds of thousands of logs which is wont to do around leader reelection time -- a process rare but run on occasion whether because a machine is dropped or happens to hiccup.
 ====
@@ -120,12 +118,11 @@ To point HBase at an existing ZooKeeper cluster, one that is not managed by HBas
 ----
 
   ...
-  # Tell HBase whether it should manage its own instance of Zookeeper or not.
+  # Tell HBase whether it should manage its own instance of ZooKeeper or not.
   export HBASE_MANAGES_ZK=false
 ----
 
-Next set ensemble locations and client port, if non-standard, in _hbase-site.xml_, or add a suitably configured _zoo.cfg_ to HBase's _CLASSPATH_.
-HBase will prefer the configuration found in _zoo.cfg_ over any settings in _hbase-site.xml_.
+Next set ensemble locations and client port, if non-standard, in _hbase-site.xml_.
 
 When HBase manages ZooKeeper, it will start/stop the ZooKeeper servers as a part of the regular start/stop scripts.
 If you would like to run ZooKeeper yourself, independent of HBase start/stop, you would do the following
@@ -141,32 +138,32 @@ Just make sure to set `HBASE_MANAGES_ZK` to `false`      if you want it to stay
 For more information about running a distinct ZooKeeper cluster, see the ZooKeeper link:http://hadoop.apache.org/zookeeper/docs/current/zookeeperStarted.html[Getting
         Started Guide].
 Additionally, see the link:http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7[ZooKeeper Wiki] or the link:http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_zkMulitServerSetup[ZooKeeper
-        documentation] for more information on ZooKeeper sizing. 
+        documentation] for more information on ZooKeeper sizing.
 
 [[zk.sasl.auth]]
 == SASL Authentication with ZooKeeper
 
-Newer releases of Apache HBase (>= 0.92) will support connecting to a ZooKeeper Quorum that supports SASL authentication (which is available in Zookeeper versions 3.4.0 or later).
+Newer releases of Apache HBase (>= 0.92) will support connecting to a ZooKeeper Quorum that supports SASL authentication (which is available in ZooKeeper versions 3.4.0 or later).
 
 This describes how to set up HBase to mutually authenticate with a ZooKeeper Quorum.
-ZooKeeper/HBase mutual authentication (link:https://issues.apache.org/jira/browse/HBASE-2418[HBASE-2418]) is required as part of a complete secure HBase configuration (link:https://issues.apache.org/jira/browse/HBASE-3025[HBASE-3025]). For simplicity of explication, this section ignores additional configuration required (Secure HDFS and Coprocessor configuration). It's recommended to begin with an HBase-managed Zookeeper configuration (as opposed to a standalone Zookeeper quorum) for ease of learning. 
+ZooKeeper/HBase mutual authentication (link:https://issues.apache.org/jira/browse/HBASE-2418[HBASE-2418]) is required as part of a complete secure HBase configuration (link:https://issues.apache.org/jira/browse/HBASE-3025[HBASE-3025]). For simplicity of explication, this section ignores additional configuration required (Secure HDFS and Coprocessor configuration). It's recommended to begin with an HBase-managed ZooKeeper configuration (as opposed to a standalone ZooKeeper quorum) for ease of learning.
 
 === Operating System Prerequisites
 
 You need to have a working Kerberos KDC setup.
 For each `$HOST` that will run a ZooKeeper server, you should have a principle `zookeeper/$HOST`.
 For each such host, add a service key (using the `kadmin` or `kadmin.local`        tool's `ktadd` command) for `zookeeper/$HOST` and copy this file to `$HOST`, and make it readable only to the user that will run zookeeper on `$HOST`.
-Note the location of this file, which we will use below as _$PATH_TO_ZOOKEEPER_KEYTAB_. 
+Note the location of this file, which we will use below as _$PATH_TO_ZOOKEEPER_KEYTAB_.
 
 Similarly, for each `$HOST` that will run an HBase server (master or regionserver), you should have a principle: `hbase/$HOST`.
 For each host, add a keytab file called _hbase.keytab_ containing a service key for `hbase/$HOST`, copy this file to `$HOST`, and make it readable only to the user that will run an HBase service on `$HOST`.
-Note the location of this file, which we will use below as _$PATH_TO_HBASE_KEYTAB_. 
+Note the location of this file, which we will use below as _$PATH_TO_HBASE_KEYTAB_.
 
 Each user who will be an HBase client should also be given a Kerberos principal.
 This principal should usually have a password assigned to it (as opposed to, as with the HBase servers, a keytab file) which only this user knows.
 The client's principal's `maxrenewlife` should be set so that it can be renewed enough so that the user can complete their HBase client processes.
 For example, if a user runs a long-running HBase client process that takes at most 3 days, we might create this user's principal within `kadmin` with: `addprinc -maxrenewlife 3days`.
-The Zookeeper client and server libraries manage their own ticket refreshment by running threads that wake up periodically to do the refreshment. 
+The ZooKeeper client and server libraries manage their own ticket refreshment by running threads that wake up periodically to do the refreshment.
 
 On each host that will run an HBase client (e.g. `hbase shell`), add the following file to the HBase home directory's _conf_ directory:
 
@@ -182,7 +179,7 @@ Client {
 
 We'll refer to this JAAS configuration file as _$CLIENT_CONF_        below.
 
-=== HBase-managed Zookeeper Configuration
+=== HBase-managed ZooKeeper Configuration
 
 On each node that will run a zookeeper, a master, or a regionserver, create a link:http://docs.oracle.com/javase/1.4.2/docs/guide/security/jgss/tutorials/LoginConfigFile.html[JAAS]        configuration file in the conf directory of the node's _HBASE_HOME_        directory that looks like the following:
 
@@ -208,10 +205,10 @@ Client {
 
 where the _$PATH_TO_HBASE_KEYTAB_ and _$PATH_TO_ZOOKEEPER_KEYTAB_ files are what you created above, and `$HOST` is the hostname for that node.
 
-The `Server` section will be used by the Zookeeper quorum server, while the `Client` section will be used by the HBase master and regionservers.
+The `Server` section will be used by the ZooKeeper quorum server, while the `Client` section will be used by the HBase master and regionservers.
 The path to this file should be substituted for the text _$HBASE_SERVER_CONF_ in the _hbase-env.sh_ listing below.
 
-The path to this file should be substituted for the text _$CLIENT_CONF_ in the _hbase-env.sh_ listing below. 
+The path to this file should be substituted for the text _$CLIENT_CONF_ in the _hbase-env.sh_ listing below.
 
 Modify your _hbase-env.sh_ to include the following:
 
@@ -256,9 +253,9 @@ Modify your _hbase-site.xml_ on each node that will run zookeeper, master or reg
 </configuration>
 ----
 
-where `$ZK_NODES` is the comma-separated list of hostnames of the Zookeeper Quorum hosts.
+where `$ZK_NODES` is the comma-separated list of hostnames of the ZooKeeper Quorum hosts.
 
-Start your hbase cluster by running one or more of the following set of commands on the appropriate hosts: 
+Start your hbase cluster by running one or more of the following set of commands on the appropriate hosts:
 
 ----
 
@@ -267,7 +264,7 @@ bin/hbase master start
 bin/hbase regionserver start
 ----
 
-=== External Zookeeper Configuration
+=== External ZooKeeper Configuration
 
 Add a JAAS configuration file that looks like:
 
@@ -312,20 +309,22 @@ Modify your _hbase-site.xml_ on each node that will run a master or regionserver
     <name>hbase.cluster.distributed</name>
     <value>true</value>
   </property>
+  <property>
+    <name>hbase.zookeeper.property.authProvider.1</name>
+    <value>org.apache.zookeeper.server.auth.SASLAuthenticationProvider</value>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.kerberos.removeHostFromPrincipal</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.kerberos.removeRealmFromPrincipal</name>
+    <value>true</value>
+  </property>
 </configuration>
 ----
 
-where `$ZK_NODES` is the comma-separated list of hostnames of the Zookeeper Quorum hosts.
-
-Add a _zoo.cfg_ for each Zookeeper Quorum host containing:
-
-[source,java]
-----
-
-authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
-kerberos.removeHostFromPrincipal=true
-kerberos.removeRealmFromPrincipal=true
-----
+where `$ZK_NODES` is the comma-separated list of hostnames of the ZooKeeper Quorum hosts.
 
 Also on each of these hosts, create a JAAS configuration file containing:
 
@@ -343,9 +342,9 @@ Server {
 ----
 
 where `$HOST` is the hostname of each Quorum host.
-We will refer to the full pathname of this file as _$ZK_SERVER_CONF_ below. 
+We will refer to the full pathname of this file as _$ZK_SERVER_CONF_ below.
 
-Start your Zookeepers on each Zookeeper Quorum host with:
+Start your ZooKeepers on each ZooKeeper Quorum host with:
 
 [source,bourne]
 ----
@@ -353,7 +352,7 @@ Start your Zookeepers on each Zookeeper Quorum host with:
 SERVER_JVMFLAGS="-Djava.security.auth.login.config=$ZK_SERVER_CONF" bin/zkServer start
 ----
 
-Start your HBase cluster by running one or more of the following set of commands on the appropriate nodes: 
+Start your HBase cluster by running one or more of the following set of commands on the appropriate nodes:
 
 ----
 
@@ -361,9 +360,9 @@ bin/hbase master start
 bin/hbase regionserver start
 ----
 
-=== Zookeeper Server Authentication Log Output
+=== ZooKeeper Server Authentication Log Output
 
-If the configuration above is successful, you should see something similar to the following in your Zookeeper server logs:
+If the configuration above is successful, you should see something similar to the following in your ZooKeeper server logs:
 
 ----
 
@@ -381,9 +380,9 @@ If the configuration above is successful, you should see something similar to th
 11/12/05 22:43:59 INFO server.ZooKeeperServer: adding SASL authorization for authorizationID: hbase
 ----
 
-=== Zookeeper Client Authentication Log Output
+=== ZooKeeper Client Authentication Log Output
 
-On the Zookeeper client side (HBase master or regionserver), you should see something similar to the following:
+On the ZooKeeper client side (HBase master or regionserver), you should see something similar to the following:
 
 ----
 
@@ -414,7 +413,7 @@ mvn clean test -Dtest=TestZooKeeperACL
 ----
 
 Then configure HBase as described above.
-Manually edit target/cached_classpath.txt (see below): 
+Manually edit target/cached_classpath.txt (see below):
 
 ----
 
@@ -438,7 +437,7 @@ mv target/tmp.txt target/cached_classpath.txt
 
 ==== Set JAAS configuration programmatically
 
-This would avoid the need for a separate Hadoop jar that fixes link:https://issues.apache.org/jira/browse/HADOOP-7070[HADOOP-7070]. 
+This would avoid the need for a separate Hadoop jar that fixes link:https://issues.apache.org/jira/browse/HADOOP-7070[HADOOP-7070].
 
 ==== Elimination of `kerberos.removeHostFromPrincipal` and`kerberos.removeRealmFromPrincipal`
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/book.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index b2bd151..e5898d5 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -61,9 +61,11 @@ include::_chapters/schema_design.adoc[]
 include::_chapters/mapreduce.adoc[]
 include::_chapters/security.adoc[]
 include::_chapters/architecture.adoc[]
+include::_chapters/hbase_mob.adoc[]
 include::_chapters/hbase_apis.adoc[]
 include::_chapters/external_apis.adoc[]
 include::_chapters/thrift_filter_language.adoc[]
+include::_chapters/spark.adoc[]
 include::_chapters/cp.adoc[]
 include::_chapters/performance.adoc[]
 include::_chapters/troubleshooting.adoc[]
@@ -71,6 +73,7 @@ include::_chapters/case_studies.adoc[]
 include::_chapters/ops_mgt.adoc[]
 include::_chapters/developer.adoc[]
 include::_chapters/unit_testing.adoc[]
+include::_chapters/protobuf.adoc[]
 include::_chapters/zookeeper.adoc[]
 include::_chapters/community.adoc[]
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/site/resources/images/hbase_logo_with_orca_large.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_logo_with_orca_large.png b/src/main/site/resources/images/hbase_logo_with_orca_large.png
new file mode 100644
index 0000000..e91eb8d
Binary files /dev/null and b/src/main/site/resources/images/hbase_logo_with_orca_large.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/site/resources/images/hbasecon2016-stack-logo.jpg
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbasecon2016-stack-logo.jpg b/src/main/site/resources/images/hbasecon2016-stack-logo.jpg
new file mode 100644
index 0000000..b59280d
Binary files /dev/null and b/src/main/site/resources/images/hbasecon2016-stack-logo.jpg differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/site/resources/images/hbasecon2016-stacked.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbasecon2016-stacked.png b/src/main/site/resources/images/hbasecon2016-stacked.png
new file mode 100644
index 0000000..4ff181e
Binary files /dev/null and b/src/main/site/resources/images/hbasecon2016-stacked.png differ


[8/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
HBASE-15347 updated asciidoc for 1.3


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6cb8a436
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6cb8a436
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6cb8a436

Branch: refs/heads/branch-1.3
Commit: 6cb8a436c3584445f8e87e0251b8174be7d9dc21
Parents: 505d48a
Author: Mikhail Antonov <an...@apache.org>
Authored: Wed Oct 26 13:07:08 2016 -0700
Committer: Mikhail Antonov <an...@apache.org>
Committed: Wed Oct 26 13:07:08 2016 -0700

----------------------------------------------------------------------
 .../asciidoc/_chapters/appendix_acl_matrix.adoc |   4 +-
 .../appendix_contributing_to_documentation.adoc | 268 +++---
 .../_chapters/appendix_hfile_format.adoc        | 176 ++--
 src/main/asciidoc/_chapters/architecture.adoc   | 423 ++++++---
 src/main/asciidoc/_chapters/asf.adoc            |   4 +-
 src/main/asciidoc/_chapters/case_studies.adoc   |   2 +-
 src/main/asciidoc/_chapters/community.adoc      |  42 +-
 src/main/asciidoc/_chapters/compression.adoc    |  84 +-
 src/main/asciidoc/_chapters/configuration.adoc  | 146 +--
 src/main/asciidoc/_chapters/cp.adoc             | 887 +++++++++++++++---
 src/main/asciidoc/_chapters/datamodel.adoc      |  11 +-
 src/main/asciidoc/_chapters/developer.adoc      | 657 +++++++------
 src/main/asciidoc/_chapters/external_apis.adoc  | 920 ++++++++++++++++++-
 src/main/asciidoc/_chapters/faq.adoc            |  24 +-
 .../asciidoc/_chapters/getting_started.adoc     |  25 +-
 src/main/asciidoc/_chapters/hbase-default.adoc  | 602 +++++-------
 src/main/asciidoc/_chapters/hbase_apis.adoc     |   8 +-
 src/main/asciidoc/_chapters/hbase_history.adoc  |   8 +-
 src/main/asciidoc/_chapters/hbase_mob.adoc      | 236 +++++
 src/main/asciidoc/_chapters/hbck_in_depth.adoc  |  24 +-
 src/main/asciidoc/_chapters/mapreduce.adoc      |  57 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 361 +++++++-
 src/main/asciidoc/_chapters/other_info.adoc     |  34 +-
 src/main/asciidoc/_chapters/performance.adoc    |  88 +-
 src/main/asciidoc/_chapters/preface.adoc        |  54 +-
 src/main/asciidoc/_chapters/protobuf.adoc       | 153 +++
 src/main/asciidoc/_chapters/rpc.adoc            |  25 +-
 src/main/asciidoc/_chapters/schema_design.adoc  | 242 ++++-
 src/main/asciidoc/_chapters/security.adoc       | 149 ++-
 src/main/asciidoc/_chapters/shell.adoc          |  64 +-
 src/main/asciidoc/_chapters/spark.adoc          | 690 ++++++++++++++
 .../_chapters/thrift_filter_language.adoc       |   3 +-
 src/main/asciidoc/_chapters/tracing.adoc        |  65 +-
 .../asciidoc/_chapters/troubleshooting.adoc     |  76 +-
 src/main/asciidoc/_chapters/unit_testing.adoc   |  74 +-
 src/main/asciidoc/_chapters/upgrading.adoc      |  33 +-
 src/main/asciidoc/_chapters/ycsb.adoc           |   1 +
 src/main/asciidoc/_chapters/zookeeper.adoc      |  85 +-
 src/main/asciidoc/book.adoc                     |   3 +
 .../images/hbase_logo_with_orca_large.png       | Bin 0 -> 21196 bytes
 .../images/hbasecon2016-stack-logo.jpg          | Bin 0 -> 32105 bytes
 .../resources/images/hbasecon2016-stacked.png   | Bin 0 -> 24924 bytes
 42 files changed, 5227 insertions(+), 1581 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
index cb285f3..e222875 100644
--- a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
+++ b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
@@ -65,7 +65,7 @@ Possible permissions include the following:
 For the most part, permissions work in an expected way, with the following caveats:
 
 Having Write permission does not imply Read permission.::
-  It is possible and sometimes desirable for a user to be able to write data that same user cannot read. One such example is a log-writing process. 
+  It is possible and sometimes desirable for a user to be able to write data that same user cannot read. One such example is a log-writing process.
 The [systemitem]+hbase:meta+ table is readable by every user, regardless of the user's other grants or restrictions.::
   This is a requirement for HBase to function correctly.
 `CheckAndPut` and `CheckAndDelete` operations will fail if the user does not have both Write and Read permission.::
@@ -100,7 +100,7 @@ In case the table goes out of date, the unit tests which check for accuracy of p
 |        | stopMaster | superuser\|global(A)
 |        | snapshot | superuser\|global(A)\|NS(A)\|TableOwner\|table(A)
 |        | listSnapshot | superuser\|global(A)\|SnapshotOwner
-|        | cloneSnapshot | superuser\|global(A)
+|        | cloneSnapshot | superuser\|global(A)\|(SnapshotOwner & TableName matches)
 |        | restoreSnapshot | superuser\|global(A)\|SnapshotOwner & (NS(A)\|TableOwner\|table(A))
 |        | deleteSnapshot | superuser\|global(A)\|SnapshotOwner
 |        | createNamespace | superuser\|global(A)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc b/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
index 6b31059..ce6f835 100644
--- a/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
+++ b/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
@@ -30,15 +30,14 @@
 :toc: left
 :source-language: java
 
-The Apache HBase project welcomes contributions to all aspects of the project, including the documentation.
+The Apache HBase project welcomes contributions to all aspects of the project,
+including the documentation.
 
 In HBase, documentation includes the following areas, and probably some others:
 
 * The link:http://hbase.apache.org/book.html[HBase Reference
   Guide] (this book)
 * The link:http://hbase.apache.org/[HBase website]
-* The link:http://wiki.apache.org/hadoop/Hbase[HBase
-  Wiki]
 * API documentation
 * Command-line utility output and help text
 * Web UI strings, explicit help text, context-sensitive strings, and others
@@ -46,99 +45,122 @@ In HBase, documentation includes the following areas, and probably some others:
 * Comments in source files, configuration files, and others
 * Localization of any of the above into target languages other than English
 
-No matter which area you want to help out with, the first step is almost always to download (typically by cloning the Git repository) and familiarize yourself with the HBase source code.
-The only exception in the list above is the HBase Wiki, which is edited online.
-For information on downloading and building the source, see <<developer,developer>>.
-
-=== Getting Access to the Wiki
-
-The HBase Wiki is not well-maintained and much of its content has been moved into the HBase Reference Guide (this guide). However, some pages on the Wiki are well maintained, and it would be great to have some volunteers willing to help out with the Wiki.
-To request access to the Wiki, register a new account at link:https://wiki.apache.org/hadoop/Hbase?action=newaccount[https://wiki.apache.org/hadoop/Hbase?action=newaccount].
-Contact one of the HBase committers, who can either give you access or refer you to someone who can.
+No matter which area you want to help out with, the first step is almost always
+to download (typically by cloning the Git repository) and familiarize yourself
+with the HBase source code. For information on downloading and building the source,
+see <<developer,developer>>.
 
 === Contributing to Documentation or Other Strings
 
-If you spot an error in a string in a UI, utility, script, log message, or elsewhere, or you think something could be made more clear, or you think text needs to be added where it doesn't currently exist, the first step is to file a JIRA.
-Be sure to set the component to `Documentation` in addition any other involved components.
-Most components have one or more default owners, who monitor new issues which come into those queues.
-Regardless of whether you feel able to fix the bug, you should still file bugs where you see them.
+If you spot an error in a string in a UI, utility, script, log message, or elsewhere,
+or you think something could be made more clear, or you think text needs to be added
+where it doesn't currently exist, the first step is to file a JIRA. Be sure to set
+the component to `Documentation` in addition any other involved components. Most
+components have one or more default owners, who monitor new issues which come into
+those queues. Regardless of whether you feel able to fix the bug, you should still
+file bugs where you see them.
 
 If you want to try your hand at fixing your newly-filed bug, assign it to yourself.
-You will need to clone the HBase Git repository to your local system and work on the issue there.
-When you have developed a potential fix, submit it for review.
-If it addresses the issue and is seen as an improvement, one of the HBase committers will commit it to one or more branches, as appropriate.
+You will need to clone the HBase Git repository to your local system and work on
+the issue there. When you have developed a potential fix, submit it for review.
+If it addresses the issue and is seen as an improvement, one of the HBase committers
+will commit it to one or more branches, as appropriate.
 
+[[submit_doc_patch_procedure]]
 .Procedure: Suggested Work flow for Submitting Patches
-This procedure goes into more detail than Git pros will need, but is included in this appendix so that people unfamiliar with Git can feel confident contributing to HBase while they learn.
+This procedure goes into more detail than Git pros will need, but is included
+in this appendix so that people unfamiliar with Git can feel confident contributing
+to HBase while they learn.
 
 . If you have not already done so, clone the Git repository locally.
   You only need to do this once.
-. Fairly often, pull remote changes into your local repository by using the `git pull` command, while your master branch is checked out.
+. Fairly often, pull remote changes into your local repository by using the
+`git pull` command, while your tracking branch is checked out.
 . For each issue you work on, create a new branch.
-  One convention that works well for naming the branches is to name a given branch the same as the JIRA it relates to:
+  One convention that works well for naming the branches is to name a given branch
+  the same as the JIRA it relates to:
 +
 ----
 $ git checkout -b HBASE-123456
 ----
 
-. Make your suggested changes on your branch, committing your changes to your local repository often.
-  If you need to switch to working on a different issue, remember to check out the appropriate branch.
-. When you are ready to submit your patch, first be sure that HBase builds cleanly and behaves as expected in your modified branch.
-  If you have made documentation changes, be sure the documentation and website builds by running `mvn clean site`.
-+
-NOTE: Before you use the `site` target the very first time, be sure you have built HBase at least once, in order to fetch all the Maven dependencies you need.
-+
-----
-$ mvn clean install -DskipTests               # Builds HBase
-----
-+
-----
-$ mvn clean site -DskipTests                  # Builds the website and documentation
-----
-+
-If any errors occur, address them.
-
-. If it takes you several days or weeks to implement your fix, or you know that the area of the code you are working in has had a lot of changes lately, make sure you rebase your branch against the remote master and take care of any conflicts before submitting your patch.
+. Make your suggested changes on your branch, committing your changes to your
+local repository often. If you need to switch to working on a different issue,
+remember to check out the appropriate branch.
+. When you are ready to submit your patch, first be sure that HBase builds cleanly
+and behaves as expected in your modified branch.
+. If you have made documentation changes, be sure the documentation and website
+builds by running `mvn clean site`.
+. If it takes you several days or weeks to implement your fix, or you know that
+the area of the code you are working in has had a lot of changes lately, make
+sure you rebase your branch against the remote master and take care of any conflicts
+before submitting your patch.
 +
 ----
-
 $ git checkout HBASE-123456
 $ git rebase origin/master
 ----
 
-. Generate your patch against the remote master.
-  Run the following command from the top level of your git repository (usually called `hbase`):
+. Generate your patch against the remote master. Run the following command from
+the top level of your git repository (usually called `hbase`):
 +
 ----
 $ git format-patch --stdout origin/master > HBASE-123456.patch
 ----
 +
 The name of the patch should contain the JIRA ID.
-Look over the patch file to be sure that you did not change any additional files by accident and that there are no other surprises.
-When you are satisfied, attach the patch to the JIRA and click the btn:[Patch Available] button.
-A reviewer will review your patch.
-If you need to submit a new version of the patch, leave the old one on the JIRA and add a version number to the name of the new patch.
-
+. Look over the patch file to be sure that you did not change any additional files
+by accident and that there are no other surprises.
+. When you are satisfied, attach the patch to the JIRA and click the
+btn:[Patch Available] button. A reviewer will review your patch.
+. If you need to submit a new version of the patch, leave the old one on the
+JIRA and add a version number to the name of the new patch.
 . After a change has been committed, there is no need to keep your local branch around.
-  Instead you should run `git pull` to get the new change into your master branch.
 
 === Editing the HBase Website
 
 The source for the HBase website is in the HBase source, in the _src/main/site/_ directory.
-Within this directory, source for the individual pages is in the _xdocs/_ directory, and images referenced in those pages are in the _images/_ directory.
+Within this directory, source for the individual pages is in the _xdocs/_ directory,
+and images referenced in those pages are in the _resources/images/_ directory.
 This directory also stores images used in the HBase Reference Guide.
 
-The website's pages are written in an HTML-like XML dialect called xdoc, which has a reference guide at link:http://maven.apache.org/archives/maven-1.x/plugins/xdoc/reference/xdocs.html.
-You can edit these files in a plain-text editor, an IDE, or an XML editor such as XML Mind XML Editor (XXE) or Oxygen XML Author. 
+The website's pages are written in an HTML-like XML dialect called xdoc, which
+has a reference guide at
+http://maven.apache.org/archives/maven-1.x/plugins/xdoc/reference/xdocs.html.
+You can edit these files in a plain-text editor, an IDE, or an XML editor such
+as XML Mind XML Editor (XXE) or Oxygen XML Author.
+
+To preview your changes, build the website using the `mvn clean site -DskipTests`
+command. The HTML output resides in the _target/site/_ directory.
+When you are satisfied with your changes, follow the procedure in
+<<submit_doc_patch_procedure,submit doc patch procedure>> to submit your patch.
 
-To preview your changes, build the website using the +mvn clean site
-                -DskipTests+ command.
-The HTML output resides in the _target/site/_ directory.
-When you are satisfied with your changes, follow the procedure in <<submit_doc_patch_procedure,submit doc patch procedure>> to submit your patch.
+[[website_publish]]
+=== Publishing the HBase Website and Documentation
+
+HBase uses the ASF's `gitpubsub` mechanism.
+. After generating the website and documentation
+artifacts using `mvn clean site site:stage`, check out the `asf-site` repository.
+
+. Remove previously-generated content using the following command:
++
+----
+rm -rf rm -rf *apidocs* *xref* *book* *.html *.pdf* css js
+----
++
+WARNING: Do not remove the `0.94/` directory. To regenerate them, you must check out
+the 0.94 branch and run `mvn clean site site:stage` from there, and then copy the
+artifacts to the 0.94/ directory of the `asf-site` branch.
+
+. Copy the contents of `target/staging` to the branch.
+
+. Add and commit your changes, and submit a patch for review.
 
 === HBase Reference Guide Style Guide and Cheat Sheet
 
-The HBase Reference Guide is written in Asciidoc and built using link:http://asciidoctor.org[AsciiDoctor]. The following cheat sheet is included for your reference. More nuanced and comprehensive documentation is available at link:http://asciidoctor.org/docs/user-manual/.
+The HBase Reference Guide is written in Asciidoc and built using link:http://asciidoctor.org[AsciiDoctor].
+The following cheat sheet is included for your reference. More nuanced and comprehensive documentation
+is available at http://asciidoctor.org/docs/user-manual/.
 
 .AsciiDoc Cheat Sheet
 [cols="1,1,a",options="header"]
@@ -147,15 +169,15 @@ The HBase Reference Guide is written in Asciidoc and built using link:http://asc
 | A paragraph | a paragraph | Just type some text with a blank line at the top and bottom.
 | Add line breaks within a paragraph without adding blank lines | Manual line breaks | This will break + at the plus sign. Or prefix the whole paragraph with a line containing '[%hardbreaks]'
 | Give a title to anything | Colored italic bold differently-sized text | .MyTitle (no space between the period and the words) on the line before the thing to be titled
-| In-Line Code or commands | monospace | \`text`  
+| In-Line Code or commands | monospace | \`text`
 | In-line literal content (things to be typed exactly as shown) | bold mono | \*\`typethis`*
 | In-line replaceable content (things to substitute with your own values) | bold italic mono | \*\_typesomething_*
-| Code blocks with highlighting | monospace, highlighted, preserve space | 
+| Code blocks with highlighting | monospace, highlighted, preserve space |
 ........
 [source,java]
----- 
-  myAwesomeCode() { 
-} 
+----
+  myAwesomeCode() {
+}
 ----
 ........
 | Code block included from a separate file | included just as though it were part of the main file |
@@ -165,51 +187,52 @@ The HBase Reference Guide is written in Asciidoc and built using link:http://asc
 include\::path/to/app.rb[]
 ----
 ................
-| Include only part of a separate file | Similar to Javadoc | See link:http://asciidoctor.org/docs/user-manual/#by-tagged-regions
+| Include only part of a separate file | Similar to Javadoc
+| See http://asciidoctor.org/docs/user-manual/#by-tagged-regions
 | Filenames, directory names, new terms | italic | \_hbase-default.xml_
-| External naked URLs | A link with the URL as link text | 
+| External naked URLs | A link with the URL as link text |
 ----
 link:http://www.google.com
 ----
 
-| External URLs with text | A link with arbitrary link text | 
+| External URLs with text | A link with arbitrary link text |
 ----
 link:http://www.google.com[Google]
 ----
 
-| Create an internal anchor to cross-reference | not rendered | 
+| Create an internal anchor to cross-reference | not rendered |
 ----
 [[anchor_name]]
 ----
-| Cross-reference an existing anchor using its default title| an internal hyperlink using the element title if available, otherwise using the anchor name | 
+| Cross-reference an existing anchor using its default title| an internal hyperlink using the element title if available, otherwise using the anchor name |
 ----
 <<anchor_name>>
 ----
-| Cross-reference an existing anchor using custom text | an internal hyperlink using arbitrary text | 
+| Cross-reference an existing anchor using custom text | an internal hyperlink using arbitrary text |
 ----
 <<anchor_name,Anchor Text>>
 ----
-| A block image | The image with alt text | 
+| A block image | The image with alt text |
 ----
-image::sunset.jpg[Alt Text] 
+image::sunset.jpg[Alt Text]
 ----
 (put the image in the src/main/site/resources/images directory)
-| An inline image | The image with alt text, as part of the text flow | 
+| An inline image | The image with alt text, as part of the text flow |
 ----
 image:sunset.jpg [Alt Text]
 ----
 (only one colon)
-| Link to a remote image | show an image hosted elsewhere | 
+| Link to a remote image | show an image hosted elsewhere |
 ----
-image::http://inkscape.org/doc/examples/tux.svg[Tux,250,350] 
+image::http://inkscape.org/doc/examples/tux.svg[Tux,250,350]
 ----
 (or `image:`)
 | Add dimensions or a URL to the image | depends | inside the brackets after the alt text, specify width, height and/or link="http://my_link.com"
-| A footnote | subscript link which takes you to the footnote | 
+| A footnote | subscript link which takes you to the footnote |
 ----
 Some text.footnote:[The footnote text.]
 ----
-| A note or warning with no title | The admonition image followed by the admonition | 
+| A note or warning with no title | The admonition image followed by the admonition |
 ----
 NOTE:My note here
 ----
@@ -217,7 +240,7 @@ NOTE:My note here
 ----
 WARNING:My warning here
 ----
-| A complex note | The note has a title and/or multiple paragraphs and/or code blocks or lists, etc | 
+| A complex note | The note has a title and/or multiple paragraphs and/or code blocks or lists, etc |
 ........
 .The Title
 [NOTE]
@@ -228,26 +251,26 @@ some source code
 ----
 ====
 ........
-| Bullet lists | bullet lists | 
+| Bullet lists | bullet lists |
 ----
 * list item 1
 ----
 (see http://asciidoctor.org/docs/user-manual/#unordered-lists)
-| Numbered lists | numbered list | 
+| Numbered lists | numbered list |
 ----
-. list item 2 
+. list item 2
 ----
 (see http://asciidoctor.org/docs/user-manual/#ordered-lists)
-| Checklists | Checked or unchecked boxes | 
+| Checklists | Checked or unchecked boxes |
 Checked:
 ----
-- [*] 
+- [*]
 ----
 Unchecked:
 ----
 - [ ]
 ----
-| Multiple levels of lists | bulleted or numbered or combo | 
+| Multiple levels of lists | bulleted or numbered or combo |
 ----
 . Numbered (1), at top level
 * Bullet (2), nested under 1
@@ -257,14 +280,18 @@ Unchecked:
 ** Bullet (6), nested under 5
 - [x] Checked (7), at top level
 ----
-| Labelled lists / variablelists | a list item title or summary followed by content | 
+| Labelled lists / variablelists | a list item title or summary followed by content |
 ----
-Title:: content 
+Title:: content
 
 Title::
   content
 ----
-| Sidebars, quotes, or other blocks of text | a block of text, formatted differently from the default | Delimited using different delimiters, see link:http://asciidoctor.org/docs/user-manual/#built-in-blocks-summary. Some of the examples above use delimiters like \...., ----,====.
+| Sidebars, quotes, or other blocks of text
+| a block of text, formatted differently from the default
+| Delimited using different delimiters,
+see http://asciidoctor.org/docs/user-manual/#built-in-blocks-summary.
+Some of the examples above use delimiters like \...., ----,====.
 ........
 [example]
 ====
@@ -288,7 +315,7 @@ ____
 ........
 
 If you want to insert literal Asciidoc content that keeps being interpreted, when in doubt, use eight dots as the delimiter at the top and bottom.
-| Nested Sections | chapter, section, sub-section, etc | 
+| Nested Sections | chapter, section, sub-section, etc |
 ----
 = Book (or chapter if the chapter can be built alone, see the leveloffset info below)
 
@@ -296,7 +323,7 @@ If you want to insert literal Asciidoc content that keeps being interpreted, whe
 
 === Section (or subsection, etc)
 
-==== Subsection 
+==== Subsection
 ----
 
 and so on up to 6 levels (think carefully about going deeper than 4 levels, maybe you can just titled paragraphs or lists instead). Note that you can include a book inside another book by adding the `:leveloffset:+1` macro directive directly before your include, and resetting it to 0 directly after. See the _book.adoc_ source for examples, as this is how this guide handles chapters. *Don't do it for prefaces, glossaries, appendixes, or other special types of chapters.*
@@ -309,7 +336,7 @@ include::[/path/to/file.adoc]
 
 For plenty of examples. see _book.adoc_.
 | A table | a table | See http://asciidoctor.org/docs/user-manual/#tables. Generally rows are separated by newlines and columns by pipes
-| Comment out a single line | A  line is skipped during rendering | 
+| Comment out a single line | A  line is skipped during rendering |
 `+//+ This line won't show up`
 | Comment out a block | A section of the file is skipped during rendering |
 ----
@@ -317,7 +344,7 @@ For plenty of examples. see _book.adoc_.
 Nothing between the slashes will show up.
 ////
 ----
-| Highlight text for review | text shows up with yellow background | 
+| Highlight text for review | text shows up with yellow background |
 ----
 Test between #hash marks# is highlighted yellow.
 ----
@@ -326,20 +353,27 @@ Test between #hash marks# is highlighted yellow.
 
 === Auto-Generated Content
 
-Some parts of the HBase Reference Guide, most notably <<config.files,config.files>>, are generated automatically, so that this area of the documentation stays in sync with the code.
-This is done by means of an XSLT transform, which you can examine in the source at _src/main/xslt/configuration_to_asciidoc_chapter.xsl_.
-This transforms the _hbase-common/src/main/resources/hbase-default.xml_            file into an Asciidoc output which can be included in the Reference Guide.
-Sometimes, it is necessary to add configuration parameters or modify their descriptions.
-Make the modifications to the source file, and they will be included in the Reference Guide when it is rebuilt.
+Some parts of the HBase Reference Guide, most notably <<config.files,config.files>>,
+are generated automatically, so that this area of the documentation stays in
+sync with the code. This is done by means of an XSLT transform, which you can examine
+in the source at _src/main/xslt/configuration_to_asciidoc_chapter.xsl_. This
+transforms the _hbase-common/src/main/resources/hbase-default.xml_ file into an
+Asciidoc output which can be included in the Reference Guide.
 
-It is possible that other types of content can and will be automatically generated from HBase source files in the future.
+Sometimes, it is necessary to add configuration parameters or modify their descriptions.
+Make the modifications to the source file, and they will be included in the
+Reference Guide when it is rebuilt.
 
+It is possible that other types of content can and will be automatically generated
+from HBase source files in the future.
 
 === Images in the HBase Reference Guide
 
-You can include images in the HBase Reference Guide. It is important to include an image title if possible, and alternate text always. 
-This allows screen readers to navigate to the image and also provides alternative text for the image.
-The following is an example of an image with a title and alternate text. Notice the double colon.
+You can include images in the HBase Reference Guide. It is important to include
+an image title if possible, and alternate text always. This allows screen readers
+to navigate to the image and also provides alternative text for the image.
+The following is an example of an image with a title and alternate text. Notice
+the double colon.
 
 [source,asciidoc]
 ----
@@ -347,42 +381,53 @@ The following is an example of an image with a title and alternate text. Notice
 image::sunset.jpg[Alt Text]
 ----
 
-Here is an example of an inline image with alternate text. Notice the single colon. Inline images cannot have titles. They are generally small images like GUI buttons.
+Here is an example of an inline image with alternate text. Notice the single colon.
+Inline images cannot have titles. They are generally small images like GUI buttons.
 
 [source,asciidoc]
 ----
 image:sunset.jpg[Alt Text]
 ----
 
-
 When doing a local build, save the image to the _src/main/site/resources/images/_ directory.
 When you link to the image, do not include the directory portion of the path.
 The image will be copied to the appropriate target location during the build of the output.
 
-When you submit a patch which includes adding an image to the HBase Reference Guide, attach the image to the JIRA.
-If the committer asks where the image should be committed, it should go into the above directory.
+When you submit a patch which includes adding an image to the HBase Reference Guide,
+attach the image to the JIRA. If the committer asks where the image should be
+committed, it should go into the above directory.
 
 === Adding a New Chapter to the HBase Reference Guide
 
-If you want to add a new chapter to the HBase Reference Guide, the easiest way is to copy an existing chapter file, rename it, and change the ID (in double brackets) and title. Chapters are located in the _src/main/asciidoc/_chapters/_ directory.
+If you want to add a new chapter to the HBase Reference Guide, the easiest way
+is to copy an existing chapter file, rename it, and change the ID (in double
+brackets) and title. Chapters are located in the _src/main/asciidoc/_chapters/_
+directory.
 
-Delete the existing content and create the new content.
-Then open the _src/main/asciidoc/book.adoc_ file, which is the main file for the HBase Reference Guide, and copy an existing `include` element to include your new chapter in the appropriate location.
-Be sure to add your new file to your Git repository before creating your patch.
+Delete the existing content and create the new content. Then open the
+_src/main/asciidoc/book.adoc_ file, which is the main file for the HBase Reference
+Guide, and copy an existing `include` element to include your new chapter in the
+appropriate location. Be sure to add your new file to your Git repository before
+creating your patch.
 
 When in doubt, check to see how other files have been included.
 
 === Common Documentation Issues
 
-The following documentation issues come up often.
-Some of these are preferences, but others can create mysterious build errors or other problems.
+The following documentation issues come up often. Some of these are preferences,
+but others can create mysterious build errors or other problems.
 
 [qanda]
 Isolate Changes for Easy Diff Review.::
-  Be careful with pretty-printing or re-formatting an entire XML file, even if the formatting has degraded over time. If you need to reformat a file, do that in a separate JIRA where you do not change any content. Be careful because some XML editors do a bulk-reformat when you open a new file, especially if you use GUI mode in the editor.
+  Be careful with pretty-printing or re-formatting an entire XML file, even if
+  the formatting has degraded over time. If you need to reformat a file, do that
+  in a separate JIRA where you do not change any content. Be careful because some
+  XML editors do a bulk-reformat when you open a new file, especially if you use
+  GUI mode in the editor.
 
 Syntax Highlighting::
-  The HBase Reference Guide uses `coderay` for syntax highlighting. To enable syntax highlighting for a given code listing, use the following type of syntax:
+  The HBase Reference Guide uses `coderay` for syntax highlighting. To enable
+  syntax highlighting for a given code listing, use the following type of syntax:
 +
 ........
 [source,xml]
@@ -391,5 +436,6 @@ Syntax Highlighting::
 ----
 ........
 +
-Several syntax types are supported. The most interesting ones for the HBase Reference Guide are `java`, `xml`, `sql`, and `bash`.
+Several syntax types are supported. The most interesting ones for the HBase
+Reference Guide are `java`, `xml`, `sql`, and `bash`.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/appendix_hfile_format.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_hfile_format.adoc b/src/main/asciidoc/_chapters/appendix_hfile_format.adoc
index b74763c..18eafe6 100644
--- a/src/main/asciidoc/_chapters/appendix_hfile_format.adoc
+++ b/src/main/asciidoc/_chapters/appendix_hfile_format.adoc
@@ -44,16 +44,16 @@ An HFile in version 1 format is structured as follows:
 .HFile V1 Format
 image::hfile.png[HFile Version 1]
 
-====  Block index format in version 1 
+====  Block index format in version 1
 
 The block index in version 1 is very straightforward.
-For each entry, it contains: 
+For each entry, it contains:
 
 . Offset (long)
 . Uncompressed size (int)
-. Key (a serialized byte array written using Bytes.writeByteArray) 
-.. Key length as a variable-length integer (VInt) 
-.. Key bytes 
+. Key (a serialized byte array written using Bytes.writeByteArray)
+.. Key length as a variable-length integer (VInt)
+.. Key bytes
 
 
 The number of entries in the block index is stored in the fixed file trailer, and has to be passed in to the method that reads the block index.
@@ -66,7 +66,7 @@ We fix this limitation in version 2, where we store on-disk block size instead o
 
 Note:  this feature was introduced in HBase 0.92
 
-==== Motivation 
+==== Motivation
 
 We found it necessary to revise the HFile format after encountering high memory usage and slow startup times caused by large Bloom filters and block indexes in the region server.
 Bloom filters can get as large as 100 MB per HFile, which adds up to 2 GB when aggregated over 20 regions.
@@ -80,7 +80,7 @@ Bloom filter blocks and index blocks (we call these "inline blocks") become inte
 
 HFile is a low-level file format by design, and it should not deal with application-specific details such as Bloom filters, which are handled at StoreFile level.
 Therefore, we call Bloom filter blocks in an HFile "inline" blocks.
-We also supply HFile with an interface to write those inline blocks. 
+We also supply HFile with an interface to write those inline blocks.
 
 Another format modification aimed at reducing the region server startup time is to use a contiguous "load-on-open" section that has to be loaded in memory at the time an HFile is being opened.
 Currently, as an HFile opens, there are separate seek operations to read the trailer, data/meta indexes, and file info.
@@ -91,57 +91,57 @@ In version 2, we seek once to read the trailer and seek again to read everything
 ==== Overview of Version 2
 
 The version of HBase introducing the above features reads both version 1 and 2 HFiles, but only writes version 2 HFiles.
-A version 2 HFile is structured as follows: 
+A version 2 HFile is structured as follows:
 
 .HFile Version 2 Structure
-image:hfilev2.png[HFile Version 2]   
+image:hfilev2.png[HFile Version 2]
 
 ==== Unified version 2 block format
 
-In the version 2 every block in the data section contains the following fields: 
-
-. 8 bytes: Block type, a sequence of bytes equivalent to version 1's "magic records". Supported block types are: 
-.. DATA \u2013 data blocks 
-.. LEAF_INDEX \u2013 leaf-level index blocks in a multi-level-block-index 
-.. BLOOM_CHUNK \u2013 Bloom filter chunks 
-.. META \u2013 meta blocks (not used for Bloom filters in version 2 anymore) 
-.. INTERMEDIATE_INDEX \u2013 intermediate-level index blocks in a multi-level blockindex 
-.. ROOT_INDEX \u2013 root>level index blocks in a multi>level block index 
-.. FILE_INFO \u2013 the ``file info'' block, a small key>value map of metadata 
-.. BLOOM_META \u2013 a Bloom filter metadata block in the load>on>open section 
+In the version 2 every block in the data section contains the following fields:
+
+. 8 bytes: Block type, a sequence of bytes equivalent to version 1's "magic records". Supported block types are:
+.. DATA \u2013 data blocks
+.. LEAF_INDEX \u2013 leaf-level index blocks in a multi-level-block-index
+.. BLOOM_CHUNK \u2013 Bloom filter chunks
+.. META \u2013 meta blocks (not used for Bloom filters in version 2 anymore)
+.. INTERMEDIATE_INDEX \u2013 intermediate-level index blocks in a multi-level blockindex
+.. ROOT_INDEX \u2013 root>level index blocks in a multi>level block index
+.. FILE_INFO \u2013 the ``file info'' block, a small key>value map of metadata
+.. BLOOM_META \u2013 a Bloom filter metadata block in the load>on>open section
 .. TRAILER \u2013 a fixed>size file trailer.
-  As opposed to the above, this is not an HFile v2 block but a fixed>size (for each HFile version) data structure 
-.. INDEX_V1 \u2013 this block type is only used for legacy HFile v1 block 
-. Compressed size of the block's data, not including the header (int). 
+  As opposed to the above, this is not an HFile v2 block but a fixed>size (for each HFile version) data structure
+.. INDEX_V1 \u2013 this block type is only used for legacy HFile v1 block
+. Compressed size of the block's data, not including the header (int).
 +
-Can be used for skipping the current data block when scanning HFile data. 
+Can be used for skipping the current data block when scanning HFile data.
 . Uncompressed size of the block's data, not including the header (int)
 +
-This is equal to the compressed size if the compression algorithm is NONE 
+This is equal to the compressed size if the compression algorithm is NONE
 . File offset of the previous block of the same type (long)
 +
-Can be used for seeking to the previous data/index block 
+Can be used for seeking to the previous data/index block
 . Compressed data (or uncompressed data if the compression algorithm is NONE).
 
 The above format of blocks is used in the following HFile sections:
 
 Scanned block section::
   The section is named so because it contains all data blocks that need to be read when an HFile is scanned sequentially.
-  Also contains leaf block index and Bloom chunk blocks. 
+  Also contains leaf block index and Bloom chunk blocks.
 Non-scanned block section::
   This section still contains unified-format v2 blocks but it does not have to be read when doing a sequential scan.
-  This section contains "meta" blocks and intermediate-level index blocks. 
+  This section contains "meta" blocks and intermediate-level index blocks.
 
-We are supporting "meta" blocks in version 2 the same way they were supported in version 1, even though we do not store Bloom filter data in these blocks anymore. 
+We are supporting "meta" blocks in version 2 the same way they were supported in version 1, even though we do not store Bloom filter data in these blocks anymore.
 
 ====  Block index in version 2
 
-There are three types of block indexes in HFile version 2, stored in two different formats (root and non-root): 
+There are three types of block indexes in HFile version 2, stored in two different formats (root and non-root):
 
 . Data index -- version 2 multi-level block index, consisting of:
-.. Version 2 root index, stored in the data block index section of the file 
-.. Optionally, version 2 intermediate levels, stored in the non%root format in   the data index section of the file. Intermediate levels can only be present if leaf level blocks are present 
-.. Optionally, version 2 leaf levels, stored in the non%root format inline with   data blocks 
+.. Version 2 root index, stored in the data block index section of the file
+.. Optionally, version 2 intermediate levels, stored in the non%root format in   the data index section of the file. Intermediate levels can only be present if leaf level blocks are present
+.. Optionally, version 2 leaf levels, stored in the non%root format inline with   data blocks
 . Meta index -- version 2 root index format only, stored in the meta index section of the file
 . Bloom index -- version 2 root index format only, stored in the ``load-on-open'' section as part of Bloom filter metadata.
 
@@ -150,19 +150,19 @@ There are three types of block indexes in HFile version 2, stored in two differe
 This format applies to:
 
 . Root level of the version 2 data index
-. Entire meta and Bloom indexes in version 2, which are always single-level. 
+. Entire meta and Bloom indexes in version 2, which are always single-level.
 
-A version 2 root index block is a sequence of entries of the following format, similar to entries of a version 1 block index, but storing on-disk size instead of uncompressed size. 
+A version 2 root index block is a sequence of entries of the following format, similar to entries of a version 1 block index, but storing on-disk size instead of uncompressed size.
 
-. Offset (long) 
+. Offset (long)
 +
-This offset may point to a data block or to a deeper>level index block. 
+This offset may point to a data block or to a deeper>level index block.
 
-. On-disk size (int) 
-. Key (a serialized byte array stored using Bytes.writeByteArray) 
+. On-disk size (int)
+. Key (a serialized byte array stored using Bytes.writeByteArray)
 +
-. Key (VInt) 
-. Key bytes 
+. Key (VInt)
+. Key bytes
 
 
 A single-level version 2 block index consists of just a single root index block.
@@ -172,13 +172,13 @@ For the data index and the meta index the number of entries is stored in the tra
 For a multi-level block index we also store the following fields in the root index block in the load-on-open section of the HFile, in addition to the data structure described above:
 
 . Middle leaf index block offset
-. Middle leaf block on-disk size (meaning the leaf index block containing the reference to the ``middle'' data block of the file) 
+. Middle leaf block on-disk size (meaning the leaf index block containing the reference to the ``middle'' data block of the file)
 . The index of the mid-key (defined below) in the middle leaf-level block.
 
 
 
 These additional fields are used to efficiently retrieve the mid-key of the HFile used in HFile splits, which we define as the first key of the block with a zero-based index of (n \u2013 1) / 2, if the total number of blocks in the HFile is n.
-This definition is consistent with how the mid-key was determined in HFile version 1, and is reasonable in general, because blocks are likely to be the same size on average, but we don't have any estimates on individual key/value pair sizes. 
+This definition is consistent with how the mid-key was determined in HFile version 1, and is reasonable in general, because blocks are likely to be the same size on average, but we don't have any estimates on individual key/value pair sizes.
 
 
 
@@ -189,52 +189,57 @@ When reading the HFile and the mid-key is requested, we retrieve the middle leaf
 ==== Non-root block index format in version 2
 
 This format applies to intermediate-level and leaf index blocks of a version 2 multi-level data block index.
-Every non-root index block is structured as follows. 
-
-. numEntries: the number of entries (int). 
-. entryOffsets: the ``secondary index'' of offsets of entries in the block, to facilitate a quick binary search on the key (numEntries + 1 int values). The last value is the total length of all entries in this index block.
-  For example, in a non-root index block with entry sizes 60, 80, 50 the ``secondary index'' will contain the following int array: {0, 60, 140, 190}.
+Every non-root index block is structured as follows.
+
+. numEntries: the number of entries (int).
+. entryOffsets: the "secondary index" of offsets of entries in the block, to facilitate
+  a quick binary search on the key (`numEntries + 1` int values). The last value
+  is the total length of all entries in this index block. For example, in a non-root
+  index block with entry sizes 60, 80, 50 the "secondary index" will contain the
+  following int array: `{0, 60, 140, 190}`.
 . Entries.
-  Each entry contains: 
+  Each entry contains:
 +
-. Offset of the block referenced by this entry in the file (long) 
-. On>disk size of the referenced block (int) 
+. Offset of the block referenced by this entry in the file (long)
+. On>disk size of the referenced block (int)
 . Key.
-  The length can be calculated from entryOffsets. 
+  The length can be calculated from entryOffsets.
 
 
 ==== Bloom filters in version 2
 
-In contrast with version 1, in a version 2 HFile Bloom filter metadata is stored in the load-on-open section of the HFile for quick startup. 
+In contrast with version 1, in a version 2 HFile Bloom filter metadata is stored in the load-on-open section of the HFile for quick startup.
 
-. A compound Bloom filter. 
+. A compound Bloom filter.
 +
-. Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom   filter version number 2 
-. The total byte size of all compound Bloom filter chunks (long) 
-. Number of hash functions (int 
-. Type of hash functions (int) 
-. The total key count inserted into the Bloom filter (long) 
-. The maximum total number of keys in the Bloom filter (long) 
-. The number of chunks (int) 
-. Comparator class used for Bloom filter keys, a UTF>8 encoded string stored   using Bytes.writeByteArray 
-. Bloom block index in the version 2 root block index format 
+. Bloom filter version = 3 (int). There used to be a DynamicByteBloomFilter class that had the Bloom   filter version number 2
+. The total byte size of all compound Bloom filter chunks (long)
+. Number of hash functions (int
+. Type of hash functions (int)
+. The total key count inserted into the Bloom filter (long)
+. The maximum total number of keys in the Bloom filter (long)
+. The number of chunks (int)
+. Comparator class used for Bloom filter keys, a UTF>8 encoded string stored   using Bytes.writeByteArray
+. Bloom block index in the version 2 root block index format
 
 
 ==== File Info format in versions 1 and 2
 
-The file info block is a serialized link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/io/HbaseMapWritable.html[HbaseMapWritable] (essentially a map from byte arrays to byte arrays) with the following keys, among others.
+The file info block is a serialized map from byte arrays to byte arrays, with the following keys, among others.
 StoreFile-level logic adds more keys to this.
 
 [cols="1,1", frame="all"]
 |===
 |hfile.LASTKEY| The last key of the file (byte array)
 |hfile.AVG_KEY_LEN| The average key length in the file (int)
-|hfile.AVG_VALUE_LEN| The average value length in the file (int)           
+|hfile.AVG_VALUE_LEN| The average value length in the file (int)
 |===
 
-File info format did not change in version 2.
-However, we moved the file info to the final section of the file, which can be loaded as one block at the time the HFile is being opened.
-Also, we do not store comparator in the version 2 file info anymore.
+In version 2, we did not change the file format, but we moved the file info to
+the final section of the file, which can be loaded as one block when the HFile
+is being opened.
+
+Also, we do not store the comparator in the version 2 file info anymore.
 Instead, we store it in the fixed file trailer.
 This is because we need to know the comparator at the time of parsing the load-on-open section of the HFile.
 
@@ -242,14 +247,15 @@ This is because we need to know the comparator at the time of parsing the load-o
 
 The following table shows common and different fields between fixed file trailers in versions 1 and 2.
 Note that the size of the trailer is different depending on the version, so it is ``fixed'' only within one version.
-However, the version is always stored as the last four-byte integer in the file. 
+However, the version is always stored as the last four-byte integer in the file.
 
 .Differences between HFile Versions 1 and 2
 [cols="1,1", frame="all"]
 |===
 | Version 1 | Version 2
 | |File info offset (long)
-| Data index offset (long)| loadOnOpenOffset (long) /The offset of the sectionthat we need toload when opening the file./
+| Data index offset (long)
+| loadOnOpenOffset (long) /The offset of the section that we need to load when opening the file./
 | | Number of data index entries (int)
 | metaIndexOffset (long) /This field is not being used by the version 1 reader, so we removed it from version 2./ | uncompressedDataIndexSize (long) /The total uncompressed size of the whole data block index, including root-level, intermediate-level, and leaf-level blocks./
 | | Number of meta index entries (int)
@@ -257,7 +263,7 @@ However, the version is always stored as the last four-byte integer in the file.
 | numEntries (int) | numEntries (long)
 | Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int) | Compression codec: 0 = LZO, 1 = GZ, 2 = NONE (int)
 | | The number of levels in the data block index (int)
-| | firstDataBlockOffset (long) /The offset of the first first data block. Used when scanning./
+| | firstDataBlockOffset (long) /The offset of the first data block. Used when scanning./
 | | lastDataBlockEnd (long) /The offset of the first byte after the last key/value data block. We don't need to go beyond this offset when scanning./
 | Version: 1 (int) | Version: 2 (int)
 |===
@@ -290,42 +296,42 @@ This optimization (implemented by the getShortMidpointKey method) is inspired by
 Note: this feature was introduced in HBase 0.98
 
 [[hfilev3.motivation]]
-==== Motivation 
+==== Motivation
 
-Version 3 of HFile makes changes needed to ease management of encryption at rest and cell-level metadata (which in turn is needed for cell-level ACLs and cell-level visibility labels). For more information see <<hbase.encryption.server,hbase.encryption.server>>, <<hbase.tags,hbase.tags>>, <<hbase.accesscontrol.configuration,hbase.accesscontrol.configuration>>, and <<hbase.visibility.labels,hbase.visibility.labels>>. 
+Version 3 of HFile makes changes needed to ease management of encryption at rest and cell-level metadata (which in turn is needed for cell-level ACLs and cell-level visibility labels). For more information see <<hbase.encryption.server,hbase.encryption.server>>, <<hbase.tags,hbase.tags>>, <<hbase.accesscontrol.configuration,hbase.accesscontrol.configuration>>, and <<hbase.visibility.labels,hbase.visibility.labels>>.
 
 [[hfilev3.overview]]
 ==== Overview
 
 The version of HBase introducing the above features reads HFiles in versions 1, 2, and 3 but only writes version 3 HFiles.
 Version 3 HFiles are structured the same as version 2 HFiles.
-For more information see <<hfilev2.overview,hfilev2.overview>>. 
+For more information see <<hfilev2.overview,hfilev2.overview>>.
 
 [[hvilev3.infoblock]]
 ==== File Info Block in Version 3
 
-Version 3 added two additional pieces of information to the reserved keys in the file info block. 
+Version 3 added two additional pieces of information to the reserved keys in the file info block.
 
 [cols="1,1", frame="all"]
 |===
 | hfile.MAX_TAGS_LEN | The maximum number of bytes needed to store the serialized tags for any single cell in this hfile (int)
  | hfile.TAGS_COMPRESSED | Does the block encoder for this hfile compress tags? (boolean). Should only be present if hfile.MAX_TAGS_LEN is also present.
-|===      
+|===
 
 When reading a Version 3 HFile the presence of `MAX_TAGS_LEN` is used to determine how to deserialize the cells within a data block.
-Therefore, consumers must read the file's info block prior to reading any data blocks. 
+Therefore, consumers must read the file's info block prior to reading any data blocks.
 
-When writing a Version 3 HFile, HBase will always include `MAX_TAGS_LEN ` when flushing the memstore to underlying filesystem and when using prefix tree encoding for data blocks, as described in <<compression,compression>>. 
+When writing a Version 3 HFile, HBase will always include `MAX_TAGS_LEN ` when flushing the memstore to underlying filesystem and when using prefix tree encoding for data blocks, as described in <<compression,compression>>.
 
 When compacting extant files, the default writer will omit `MAX_TAGS_LEN` if all of the files selected do not themselves contain any cells with tags.
 
-See <<compaction,compaction>> for details on the compaction file selection algorithm. 
+See <<compaction,compaction>> for details on the compaction file selection algorithm.
 
 [[hfilev3.datablock]]
 ==== Data Blocks in Version 3
 
 Within an HFile, HBase cells are stored in data blocks as a sequence of KeyValues (see <<hfilev1.overview,hfilev1.overview>>, or link:http://www.larsgeorge.com/2009/10/hbase-architecture-101-storage.html[Lars George's
-        excellent introduction to HBase Storage]). In version 3, these KeyValue optionally will include a set of 0 or more tags: 
+        excellent introduction to HBase Storage]). In version 3, these KeyValue optionally will include a set of 0 or more tags:
 
 [cols="1,1", frame="all"]
 |===
@@ -335,14 +341,14 @@ Within an HFile, HBase cells are stored in data blocks as a sequence of KeyValue
 2+| Key bytes (variable)
 2+| Value bytes (variable)
 | | Tags Length (2 bytes)
-| | Tags bytes (variable)                
-|===      
+| | Tags bytes (variable)
+|===
 
 If the info block for a given HFile contains an entry for `MAX_TAGS_LEN` each cell will have the length of that cell's tags included, even if that length is zero.
-The actual tags are stored as a sequence of tag length (2 bytes), tag type (1 byte), tag bytes (variable). The format an individual tag's bytes depends on the tag type. 
+The actual tags are stored as a sequence of tag length (2 bytes), tag type (1 byte), tag bytes (variable). The format an individual tag's bytes depends on the tag type.
 
 Note that the dependence on the contents of the info block implies that prior to reading any data blocks you must first process a file's info block.
-It also implies that prior to writing a data block you must know if the file's info block will include `MAX_TAGS_LEN`. 
+It also implies that prior to writing a data block you must know if the file's info block will include `MAX_TAGS_LEN`.
 
 [[hfilev3.fixedtrailer]]
 ==== Fixed File Trailer in Version 3
@@ -350,6 +356,6 @@ It also implies that prior to writing a data block you must know if the file's i
 The fixed file trailers written with HFile version 3 are always serialized with protocol buffers.
 Additionally, it adds an optional field to the version 2 protocol buffer named encryption_key.
 If HBase is configured to encrypt HFiles this field will store a data encryption key for this particular HFile, encrypted with the current cluster master key using AES.
-For more information see <<hbase.encryption.server,hbase.encryption.server>>. 
+For more information see <<hbase.encryption.server,hbase.encryption.server>>.
 
 :numbered:


[3/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/hbase_apis.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase_apis.adoc b/src/main/asciidoc/_chapters/hbase_apis.adoc
index 6d2777b..f27c9dc 100644
--- a/src/main/asciidoc/_chapters/hbase_apis.adoc
+++ b/src/main/asciidoc/_chapters/hbase_apis.adoc
@@ -43,8 +43,6 @@ See <<external_apis>> for more information.
 ----
 package com.example.hbase.admin;
 
-package util;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -77,7 +75,7 @@ public class Example {
          Admin admin = connection.getAdmin()) {
 
       HTableDescriptor table = new HTableDescriptor(TableName.valueOf(TABLE_NAME));
-      table.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.SNAPPY));
+      table.addFamily(new HColumnDescriptor(CF_DEFAULT).setCompressionType(Algorithm.NONE));
 
       System.out.print("Creating table. ");
       createOrOverwrite(admin, table);
@@ -90,12 +88,12 @@ public class Example {
          Admin admin = connection.getAdmin()) {
 
       TableName tableName = TableName.valueOf(TABLE_NAME);
-      if (admin.tableExists(tableName)) {
+      if (!admin.tableExists(tableName)) {
         System.out.println("Table does not exist.");
         System.exit(-1);
       }
 
-      HTableDescriptor table = new HTableDescriptor(tableName);
+      HTableDescriptor table = admin.getTableDescriptor(tableName);
 
       // Update existing table
       HColumnDescriptor newColumn = new HColumnDescriptor("NEWCF");

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/hbase_history.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase_history.adoc b/src/main/asciidoc/_chapters/hbase_history.adoc
index de4aff5..7308b90 100644
--- a/src/main/asciidoc/_chapters/hbase_history.adoc
+++ b/src/main/asciidoc/_chapters/hbase_history.adoc
@@ -29,9 +29,9 @@
 :icons: font
 :experimental:
 
-* 2006:  link:http://research.google.com/archive/bigtable.html[BigTable] paper published by Google. 
-* 2006 (end of year):  HBase development starts. 
-* 2008:  HBase becomes Hadoop sub-project. 
-* 2010:  HBase becomes Apache top-level project. 
+* 2006:  link:http://research.google.com/archive/bigtable.html[BigTable] paper published by Google.
+* 2006 (end of year):  HBase development starts.
+* 2008:  HBase becomes Hadoop sub-project.
+* 2010:  HBase becomes Apache top-level project.
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/hbase_mob.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase_mob.adoc b/src/main/asciidoc/_chapters/hbase_mob.adoc
new file mode 100644
index 0000000..3f67181
--- /dev/null
+++ b/src/main/asciidoc/_chapters/hbase_mob.adoc
@@ -0,0 +1,236 @@
+////
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+////
+
+[[hbase_mob]]
+== Storing Medium-sized Objects (MOB)
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+:toc: left
+:source-language: java
+
+Data comes in many sizes, and saving all of your data in HBase, including binary
+data such as images and documents, is ideal. While HBase can technically handle
+binary objects with cells that are larger than 100 KB in size, HBase's normal
+read and write paths are optimized for values smaller than 100KB in size. When
+HBase deals with large numbers of objects over this threshold, referred to here
+as medium objects, or MOBs, performance is degraded due to write amplification
+caused by splits and compactions. When using MOBs, ideally your objects will be between
+100KB and 10MB. HBase ***FIX_VERSION_NUMBER*** adds support
+for better managing large numbers of MOBs while maintaining performance,
+consistency, and low operational overhead. MOB support is provided by the work
+done in link:https://issues.apache.org/jira/browse/HBASE-11339[HBASE-11339]. To
+take advantage of MOB, you need to use <<hfilev3,HFile version 3>>. Optionally,
+configure the MOB file reader's cache settings for each RegionServer (see
+<<mob.cache.configure>>), then configure specific columns to hold MOB data.
+Client code does not need to change to take advantage of HBase MOB support. The
+feature is transparent to the client.
+
+=== Configuring Columns for MOB
+
+You can configure columns to support MOB during table creation or alteration,
+either in HBase Shell or via the Java API. The two relevant properties are the
+boolean `IS_MOB` and the `MOB_THRESHOLD`, which is the number of bytes at which
+an object is considered to be a MOB. Only `IS_MOB` is required. If you do not
+specify the `MOB_THRESHOLD`, the default threshold value of 100 KB is used.
+
+.Configure a Column for MOB Using HBase Shell
+====
+----
+hbase> create 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400}
+hbase> alter 't1', {NAME => 'f1', IS_MOB => true, MOB_THRESHOLD => 102400}
+----
+====
+
+.Configure a Column for MOB Using the Java API
+====
+[source,java]
+----
+...
+HColumnDescriptor hcd = new HColumnDescriptor(\u201cf\u201d);
+hcd.setMobEnabled(true);
+...
+hcd.setMobThreshold(102400L);
+...
+----
+====
+
+
+=== Testing MOB
+
+The utility `org.apache.hadoop.hbase.IntegrationTestIngestMOB` is provided to assist with testing
+the MOB feature. The utility is run as follows:
+[source,bash]
+----
+$ sudo -u hbase hbase org.apache.hadoop.hbase.IntegrationTestIngestMOB \
+            -threshold 102400 \
+            -minMobDataSize 512 \
+            -maxMobDataSize 5120
+----
+
+* `*threshold*` is the threshold at which cells are considered to be MOBs.
+   The default is 1 kB, expressed in bytes.
+* `*minMobDataSize*` is the minimum value for the size of MOB data.
+   The default is 512 B, expressed in bytes.
+* `*maxMobDataSize*` is the maximum value for the size of MOB data.
+   The default is 5 kB, expressed in bytes.
+
+
+[[mob.cache.configure]]
+=== Configuring the MOB Cache
+
+
+Because there can be a large number of MOB files at any time, as compared to the number of HFiles,
+MOB files are not always kept open. The MOB file reader cache is a LRU cache which keeps the most
+recently used MOB files open. To configure the MOB file reader's cache on each RegionServer, add
+the following properties to the RegionServer's `hbase-site.xml`, customize the configuration to
+suit your environment, and restart or rolling restart the RegionServer.
+
+.Example MOB Cache Configuration
+====
+[source,xml]
+----
+<property>
+    <name>hbase.mob.file.cache.size</name>
+    <value>1000</value>
+    <description>
+      Number of opened file handlers to cache.
+      A larger value will benefit reads by providing more file handlers per mob
+      file cache and would reduce frequent file opening and closing.
+      However, if this is set too high, this could lead to a "too many opened file handers"
+      The default value is 1000.
+    </description>
+</property>
+<property>
+    <name>hbase.mob.cache.evict.period</name>
+    <value>3600</value>
+    <description>
+      The amount of time in seconds after which an unused file is evicted from the
+      MOB cache. The default value is 3600 seconds.
+    </description>
+</property>
+<property>
+    <name>hbase.mob.cache.evict.remain.ratio</name>
+    <value>0.5f</value>
+    <description>
+      A multiplier (between 0.0 and 1.0), which determines how many files remain cached
+      after the threshold of files that remains cached after a cache eviction occurs
+      which is triggered by reaching the `hbase.mob.file.cache.size` threshold.
+      The default value is 0.5f, which means that half the files (the least-recently-used
+      ones) are evicted.
+    </description>
+</property>
+----
+====
+
+=== MOB Optimization Tasks
+
+==== Manually Compacting MOB Files
+
+To manually compact MOB files, rather than waiting for the
+<<mob.cache.configure,configuration>> to trigger compaction, use the
+`compact_mob` or `major_compact_mob` HBase shell commands. These commands
+require the first argument to be the table name, and take an optional column
+family as the second argument. If the column family is omitted, all MOB-enabled
+column families are compacted.
+
+----
+hbase> compact_mob 't1', 'c1'
+hbase> compact_mob 't1'
+hbase> major_compact_mob 't1', 'c1'
+hbase> major_compact_mob 't1'
+----
+
+These commands are also available via `Admin.compactMob` and
+`Admin.majorCompactMob` methods.
+
+==== MOB Sweeper
+
+HBase MOB a MapReduce job called the Sweeper tool for
+optimization. The Sweeper tool coalesces small MOB files or MOB files with many
+deletions or updates. The Sweeper tool is not required if you use native MOB compaction, which
+does not rely on MapReduce.
+
+To configure the Sweeper tool, set the following options:
+
+[source,xml]
+----
+<property>
+    <name>hbase.mob.sweep.tool.compaction.ratio</name>
+    <value>0.5f</value>
+    <description>
+      If there are too many cells deleted in a mob file, it's regarded
+      as an invalid file and needs to be merged.
+      If existingCellsSize/mobFileSize is less than ratio, it's regarded
+      as an invalid file. The default value is 0.5f.
+    </description>
+</property>
+<property>
+    <name>hbase.mob.sweep.tool.compaction.mergeable.size</name>
+    <value>134217728</value>
+    <description>
+      If the size of a mob file is less than this value, it's regarded as a small
+      file and needs to be merged. The default value is 128MB.
+    </description>
+</property>
+<property>
+    <name>hbase.mob.sweep.tool.compaction.memstore.flush.size</name>
+    <value>134217728</value>
+    <description>
+      The flush size for the memstore used by sweep job. Each sweep reducer owns such a memstore.
+      The default value is 128MB.
+    </description>
+</property>
+<property>
+    <name>hbase.master.mob.ttl.cleaner.period</name>
+    <value>86400</value>
+    <description>
+      The period that ExpiredMobFileCleanerChore runs. The unit is second.
+      The default value is one day.
+    </description>
+</property>
+----
+
+Next, add the HBase install directory, _`$HBASE_HOME`/*_, and HBase library directory to
+_yarn-site.xml_ Adjust this example to suit your environment.
+[source,xml]
+----
+<property>
+    <description>Classpath for typical applications.</description>
+    <name>yarn.application.classpath</name>
+    <value>
+        $HADOOP_CONF_DIR,
+        $HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,
+        $HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,
+        $HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,
+        $HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*,
+        $HBASE_HOME/*, $HBASE_HOME/lib/*
+    </value>
+</property>
+----
+
+Finally, run the `sweeper` tool for each column which is configured for MOB.
+[source,bash]
+----
+$ org.apache.hadoop.hbase.mob.compactions.Sweeper _tableName_ _familyName_
+----

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/hbck_in_depth.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbck_in_depth.adoc b/src/main/asciidoc/_chapters/hbck_in_depth.adoc
index 1b30c59..1e1f9fb 100644
--- a/src/main/asciidoc/_chapters/hbck_in_depth.adoc
+++ b/src/main/asciidoc/_chapters/hbck_in_depth.adoc
@@ -29,7 +29,7 @@
 :experimental:
 
 HBaseFsck (hbck) is a tool for checking for region consistency and table integrity problems and repairing a corrupted HBase.
-It works in two basic modes -- a read-only inconsistency identifying mode and a multi-phase read-write repair mode. 
+It works in two basic modes -- a read-only inconsistency identifying mode and a multi-phase read-write repair mode.
 
 === Running hbck to identify inconsistencies
 
@@ -42,10 +42,10 @@ $ ./bin/hbase hbck
 ----
 
 At the end of the commands output it prints OK or tells you the number of INCONSISTENCIES present.
-You may also want to run run hbck a few times because some inconsistencies can be transient (e.g.
+You may also want to run hbck a few times because some inconsistencies can be transient (e.g.
 cluster is starting up or a region is splitting). Operationally you may want to run hbck regularly and setup alert (e.g.
 via nagios) if it repeatedly reports inconsistencies . A run of hbck will report a list of inconsistencies along with a brief description of the regions and tables affected.
-The using the `-details` option will report more details including a representative listing of all the splits present in all the tables. 
+The using the `-details` option will report more details including a representative listing of all the splits present in all the tables.
 
 [source,bourne]
 ----
@@ -66,9 +66,9 @@ $ ./bin/hbase hbck TableFoo TableBar
 === Inconsistencies
 
 If after several runs, inconsistencies continue to be reported, you may have encountered a corruption.
-These should be rare, but in the event they occur newer versions of HBase include the hbck tool enabled with automatic repair options. 
+These should be rare, but in the event they occur newer versions of HBase include the hbck tool enabled with automatic repair options.
 
-There are two invariants that when violated create inconsistencies in HBase: 
+There are two invariants that when violated create inconsistencies in HBase:
 
 * HBase's region consistency invariant is satisfied if every region is assigned and deployed on exactly one region server, and all places where this state kept is in accordance.
 * HBase's table integrity invariant is satisfied if for each table, every possible row key resolves to exactly one region.
@@ -77,20 +77,20 @@ Repairs generally work in three phases -- a read-only information gathering phas
 Starting from version 0.90.0, hbck could detect region consistency problems report on a subset of possible table integrity problems.
 It also included the ability to automatically fix the most common inconsistency, region assignment and deployment consistency problems.
 This repair could be done by using the `-fix` command line option.
-These problems close regions if they are open on the wrong server or on multiple region servers and also assigns regions to region servers if they are not open. 
+These problems close regions if they are open on the wrong server or on multiple region servers and also assigns regions to region servers if they are not open.
 
 Starting from HBase versions 0.90.7, 0.92.2 and 0.94.0, several new command line options are introduced to aid repairing a corrupted HBase.
-This hbck sometimes goes by the nickname ``uberhbck''. Each particular version of uber hbck is compatible with the HBase's of the same major version (0.90.7 uberhbck can repair a 0.90.4). However, versions <=0.90.6 and versions <=0.92.1 may require restarting the master or failing over to a backup master. 
+This hbck sometimes goes by the nickname ``uberhbck''. Each particular version of uber hbck is compatible with the HBase's of the same major version (0.90.7 uberhbck can repair a 0.90.4). However, versions <=0.90.6 and versions <=0.92.1 may require restarting the master or failing over to a backup master.
 
 === Localized repairs
 
 When repairing a corrupted HBase, it is best to repair the lowest risk inconsistencies first.
 These are generally region consistency repairs -- localized single region repairs, that only modify in-memory data, ephemeral zookeeper data, or patch holes in the META table.
 Region consistency requires that the HBase instance has the state of the region's data in HDFS (.regioninfo files), the region's row in the hbase:meta table., and region's deployment/assignments on region servers and the master in accordance.
-Options for repairing region consistency include: 
+Options for repairing region consistency include:
 
 * `-fixAssignments` (equivalent to the 0.90 `-fix` option) repairs unassigned, incorrectly assigned or multiply assigned regions.
-* `-fixMeta` which removes meta rows when corresponding regions are not present in HDFS and adds new meta rows if they regions are present in HDFS while not in META.                To fix deployment and assignment problems you can run this command: 
+* `-fixMeta` which removes meta rows when corresponding regions are not present in HDFS and adds new meta rows if they regions are present in HDFS while not in META.                To fix deployment and assignment problems you can run this command:
 
 [source,bourne]
 ----
@@ -177,7 +177,7 @@ $ ./bin/hbase hbck -fixMetaOnly -fixAssignments
 ==== Special cases: HBase version file is missing
 
 HBase's data on the file system requires a version file in order to start.
-If this flie is missing, you can use the `-fixVersionFile` option to fabricating a new HBase version file.
+If this file is missing, you can use the `-fixVersionFile` option to fabricating a new HBase version file.
 This assumes that the version of hbck you are running is the appropriate version for the HBase cluster.
 
 ==== Special case: Root and META are corrupt.
@@ -205,8 +205,8 @@ However, there could be some lingering offline split parents sometimes.
 They are in META, in HDFS, and not deployed.
 But HBase can't clean them up.
 In this case, you can use the `-fixSplitParents` option to reset them in META to be online and not split.
-Therefore, hbck can merge them with other regions if fixing overlapping regions option is used. 
+Therefore, hbck can merge them with other regions if fixing overlapping regions option is used.
 
-This option should not normally be used, and it is not in `-fixAll`. 
+This option should not normally be used, and it is not in `-fixAll`.
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/mapreduce.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/mapreduce.adoc b/src/main/asciidoc/_chapters/mapreduce.adoc
index 0792c04..dfa843a 100644
--- a/src/main/asciidoc/_chapters/mapreduce.adoc
+++ b/src/main/asciidoc/_chapters/mapreduce.adoc
@@ -33,7 +33,9 @@ A good place to get started with MapReduce is http://hadoop.apache.org/docs/r2.6
 MapReduce version 2 (MR2)is now part of link:http://hadoop.apache.org/docs/r2.3.0/hadoop-yarn/hadoop-yarn-site/[YARN].
 
 This chapter discusses specific configuration steps you need to take to use MapReduce on data within HBase.
-In addition, it discusses other interactions and issues between HBase and MapReduce jobs.
+In addition, it discusses other interactions and issues between HBase and MapReduce
+jobs. Finally, it discusses <<cascading,Cascading>>, an
+link:http://www.cascading.org/[alternative API] for MapReduce.
 
 .`mapred` and `mapreduce`
 [NOTE]
@@ -63,7 +65,7 @@ The dependencies only need to be available on the local `CLASSPATH`.
 The following example runs the bundled HBase link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter] MapReduce job against a table named `usertable`.
 If you have not set the environment variables expected in the command (the parts prefixed by a `$` sign and surrounded by curly braces), you can use the actual system paths instead.
 Be sure to use the correct version of the HBase JAR for your system.
-The backticks (``` symbols) cause ths shell to execute the sub-commands, setting the output of `hbase classpath` (the command to dump HBase CLASSPATH) to `HADOOP_CLASSPATH`.
+The backticks (``` symbols) cause the shell to execute the sub-commands, setting the output of `hbase classpath` (the command to dump HBase CLASSPATH) to `HADOOP_CLASSPATH`.
 This example assumes you use a BASH-compatible shell.
 
 [source,bash]
@@ -277,7 +279,7 @@ That is where the logic for map-task assignment resides.
 
 The following is an example of using HBase as a MapReduce source in read-only manner.
 Specifically, there is a Mapper instance but no Reducer, and nothing is being emitted from the Mapper.
-There job would be defined as follows...
+The job would be defined as follows...
 
 [source,java]
 ----
@@ -590,7 +592,54 @@ public class MyMapper extends TableMapper<Text, LongWritable> {
 == Speculative Execution
 
 It is generally advisable to turn off speculative execution for MapReduce jobs that use HBase as a source.
-This can either be done on a per-Job basis through properties, on on the entire cluster.
+This can either be done on a per-Job basis through properties, or on the entire cluster.
 Especially for longer running jobs, speculative execution will create duplicate map-tasks which will double-write your data to HBase; this is probably not what you want.
 
 See <<spec.ex,spec.ex>> for more information.
+
+[[cascading]]
+== Cascading
+
+link:http://www.cascading.org/[Cascading] is an alternative API for MapReduce, which
+actually uses MapReduce, but allows you to write your MapReduce code in a simplified
+way.
+
+The following example shows a Cascading `Flow` which "sinks" data into an HBase cluster. The same
+`hBaseTap` API could be used to "source" data as well.
+
+[source, java]
+----
+// read data from the default filesystem
+// emits two fields: "offset" and "line"
+Tap source = new Hfs( new TextLine(), inputFileLhs );
+
+// store data in an HBase cluster
+// accepts fields "num", "lower", and "upper"
+// will automatically scope incoming fields to their proper familyname, "left" or "right"
+Fields keyFields = new Fields( "num" );
+String[] familyNames = {"left", "right"};
+Fields[] valueFields = new Fields[] {new Fields( "lower" ), new Fields( "upper" ) };
+Tap hBaseTap = new HBaseTap( "multitable", new HBaseScheme( keyFields, familyNames, valueFields ), SinkMode.REPLACE );
+
+// a simple pipe assembly to parse the input into fields
+// a real app would likely chain multiple Pipes together for more complex processing
+Pipe parsePipe = new Each( "insert", new Fields( "line" ), new RegexSplitter( new Fields( "num", "lower", "upper" ), " " ) );
+
+// "plan" a cluster executable Flow
+// this connects the source Tap and hBaseTap (the sink Tap) to the parsePipe
+Flow parseFlow = new FlowConnector( properties ).connect( source, hBaseTap, parsePipe );
+
+// start the flow, and block until complete
+parseFlow.complete();
+
+// open an iterator on the HBase table we stuffed data into
+TupleEntryIterator iterator = parseFlow.openSink();
+
+while(iterator.hasNext())
+  {
+  // print out each tuple from HBase
+  System.out.println( "iterator.next() = " + iterator.next() );
+  }
+
+iterator.close();
+----

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 2ec3200..a580bf8 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -57,7 +57,7 @@ Some commands take arguments. Pass no args or -h for usage.
   upgrade         Upgrade hbase
   master          Run an HBase HMaster node
   regionserver    Run an HBase HRegionServer node
-  zookeeper       Run a Zookeeper server
+  zookeeper       Run a ZooKeeper server
   rest            Run an HBase REST server
   thrift          Run the HBase Thrift server
   thrift2         Run the HBase Thrift2 server
@@ -201,7 +201,7 @@ $ ${HBASE_HOME}/bin/hbase canary -t 600000
 
 By default, the canary tool only check the read operations, it's hard to find the problem in the
 write path. To enable the write sniffing, you can run canary with the `-writeSniffing` option.
-When the write sniffing is enabled, the canary tool will create a hbase table and make sure the
+When the write sniffing is enabled, the canary tool will create an hbase table and make sure the
 regions of the table distributed on all region servers. In each sniffing period, the canary will
 try to put data to these regions to check the write availability of each region server.
 ----
@@ -304,12 +304,15 @@ The following utilities are available:
 `RowCounter`::
   Count rows in an HBase table.
 
+`CellCounter`::
+  Count cells in an HBase table.
+
 `replication.VerifyReplication`::
   Compare the data from tables in two different clusters.
   WARNING: It doesn't work for incrementColumnValues'd cells since the timestamp is changed.
   Note that this command is in a different package than the others.
 
-Each command except `RowCounter` accepts a single `--help` argument to print usage instructions.
+Each command except `RowCounter` and `CellCounter` accept a single `--help` argument to print usage instructions.
 
 [[hbck]]
 === HBase `hbck`
@@ -363,7 +366,7 @@ You can invoke it via the HBase cli with the 'wal' command.
 [NOTE]
 ====
 Prior to version 2.0, the WAL Pretty Printer was called the `HLogPrettyPrinter`, after an internal name for HBase's write ahead log.
-In those versions, you can pring the contents of a WAL using the same configuration as above, but with the 'hlog' command.
+In those versions, you can print the contents of a WAL using the same configuration as above, but with the 'hlog' command.
 
 ----
  $ ./bin/hbase hlog hdfs://example.org:8020/hbase/.logs/example.org,60020,1283516293161/10.10.21.10%3A60020.1283973724012
@@ -375,6 +378,7 @@ In those versions, you can pring the contents of a WAL using the same configurat
 
 See <<compression.test,compression.test>>.
 
+[[copy.table]]
 === CopyTable
 
 CopyTable is a utility that can copy part or of all of a table, either to the same cluster or another cluster.
@@ -433,9 +437,10 @@ Caching for the input Scan is configured via `hbase.client.scanner.caching`
 By default, CopyTable utility only copies the latest version of row cells unless `--versions=n` is explicitly specified in the command.
 ====
 
-See Jonathan Hsieh's link:http://www.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
+See Jonathan Hsieh's link:https://blog.cloudera.com/blog/2012/06/online-hbase-backups-with-copytable-2/[Online
           HBase Backups with CopyTable] blog post for more on `CopyTable`.
 
+[[export]]
 === Export
 
 Export is a utility that will dump the contents of table to HDFS in a sequence file.
@@ -445,10 +450,14 @@ Invoke via:
 $ bin/hbase org.apache.hadoop.hbase.mapreduce.Export <tablename> <outputdir> [<versions> [<starttime> [<endtime>]]]
 ----
 
+NOTE: To see usage instructions, run the command with no options. Available options include
+specifying column families and applying filters during the export.
+
 By default, the `Export` tool only exports the newest version of a given cell, regardless of the number of versions stored. To export more than one version, replace *_<versions>_* with the desired number of versions.
 
 Note: caching for the input Scan is configured via `hbase.client.scanner.caching` in the job configuration.
 
+[[import]]
 === Import
 
 Import is a utility that will load data that has been exported back into HBase.
@@ -458,12 +467,15 @@ Invoke via:
 $ bin/hbase org.apache.hadoop.hbase.mapreduce.Import <tablename> <inputdir>
 ----
 
+NOTE: To see usage instructions, run the command with no options.
+
 To import 0.94 exported files in a 0.96 cluster or onwards, you need to set system property "hbase.import.version" when running the import command as below:
 
 ----
 $ bin/hbase -Dhbase.import.version=0.94 org.apache.hadoop.hbase.mapreduce.Import <tablename> <inputdir>
 ----
 
+[[importtsv]]
 === ImportTsv
 
 ImportTsv is a utility that will load data in TSV format into HBase.
@@ -535,7 +547,7 @@ row9	c1	c2
 row10	c1	c2
 ----
 
-For ImportTsv to use this imput file, the command line needs to look like this:
+For ImportTsv to use this input file, the command line needs to look like this:
 
 ----
 
@@ -555,6 +567,7 @@ If you have preparing a lot of data for bulk loading, make sure the target HBase
 
 For more information about bulk-loading HFiles into HBase, see <<arch.bulk.load,arch.bulk.load>>
 
+[[completebulkload]]
 === CompleteBulkLoad
 
 The `completebulkload` utility will move generated StoreFiles into an HBase table.
@@ -609,7 +622,8 @@ To NOT run WALPlayer as a mapreduce job on your cluster, force it to run all in
 
 link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html[RowCounter]        is a mapreduce job to count all the rows of a table.
 This is a good utility to use as a sanity check to ensure that HBase can read all the blocks of a table if there are any concerns of metadata inconsistency.
-It will run the mapreduce all in a single process but it will run faster if you have a MapReduce cluster in place for it to exploit.
+It will run the mapreduce all in a single process but it will run faster if you have a MapReduce cluster in place for it to exploit. It is also possible to limit
+the time range of data to be scanned by using the `--starttime=[starttime]` and `--endtime=[endtime]` flags.
 
 ----
 $ bin/hbase org.apache.hadoop.hbase.mapreduce.RowCounter <tablename> [<column1> <column2>...]
@@ -632,6 +646,8 @@ The statistics gathered by RowCounter are more fine-grained and include:
 
 The program allows you to limit the scope of the run.
 Provide a row regex or prefix to limit the rows to analyze.
+Specify a time range to scan the table by using the `--starttime=[starttime]` and `--endtime=[endtime]` flags.
+
 Use `hbase.mapreduce.scan.column.family` to specify scanning a single column family.
 
 ----
@@ -649,10 +665,14 @@ See link:https://issues.apache.org/jira/browse/HBASE-4391[HBASE-4391 Add ability
 [[compaction.tool]]
 === Offline Compaction Tool
 
-See the usage for the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[Compaction
-          Tool].
-Run it like this +./bin/hbase
-          org.apache.hadoop.hbase.regionserver.CompactionTool+
+See the usage for the
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/CompactionTool.html[CompactionTool].
+Run it like:
+
+[source, bash]
+----
+$ ./bin/hbase org.apache.hadoop.hbase.regionserver.CompactionTool
+----
 
 === `hbase clean`
 
@@ -789,7 +809,7 @@ To decommission a loaded RegionServer, run the following: +$
 ====
 The `HOSTNAME` passed to _graceful_stop.sh_ must match the hostname that hbase is using to identify RegionServers.
 Check the list of RegionServers in the master UI for how HBase is referring to servers.
-Its usually hostname but can also be FQDN.
+It's usually hostname but can also be FQDN.
 Whatever HBase is using, this is what you should pass the _graceful_stop.sh_ decommission script.
 If you pass IPs, the script is not yet smart enough to make a hostname (or FQDN) of it and so it will fail when it checks if server is currently running; the graceful unloading of regions will not run.
 ====
@@ -799,6 +819,7 @@ It will verify the region deployed in the new location before it will moves the
 At this point, the _graceful_stop.sh_ tells the RegionServer `stop`.
 The master will at this point notice the RegionServer gone but all regions will have already been redeployed and because the RegionServer went down cleanly, there will be no WAL logs to split.
 
+[[lb]]
 .Load Balancer
 [NOTE]
 ====
@@ -829,12 +850,12 @@ Hence, it is better to manage the balancer apart from `graceful_stop` reenabling
 [[draining.servers]]
 ==== Decommissioning several Regions Servers concurrently
 
-If you have a large cluster, you may want to decommission more than one machine at a time by gracefully stopping mutiple RegionServers concurrently.
+If you have a large cluster, you may want to decommission more than one machine at a time by gracefully stopping multiple RegionServers concurrently.
 To gracefully drain multiple regionservers at the same time, RegionServers can be put into a "draining" state.
 This is done by marking a RegionServer as a draining node by creating an entry in ZooKeeper under the _hbase_root/draining_ znode.
 This znode has format `name,port,startcode` just like the regionserver entries under _hbase_root/rs_ znode.
 
-Without this facility, decommissioning mulitple nodes may be non-optimal because regions that are being drained from one region server may be moved to other regionservers that are also draining.
+Without this facility, decommissioning multiple nodes may be non-optimal because regions that are being drained from one region server may be moved to other regionservers that are also draining.
 Marking RegionServers to be in the draining state prevents this from happening.
 See this link:http://inchoate-clatter.blogspot.com/2012/03/hbase-ops-automation.html[blog
             post] for more details.
@@ -982,6 +1003,7 @@ Apart from resulting in higher latency, it may also be able to use all of your n
 For practical purposes, consider that a standard 1GigE NIC won't be able to read much more than _100MB/s_.
 In this case, or if you are in a OLAP environment and require having locality, then it is recommended to major compact the moved regions.
 
+[[hbase_metrics]]
 == HBase Metrics
 
 HBase emits metrics which adhere to the link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[Hadoop metrics] API.
@@ -999,7 +1021,7 @@ To configure metrics for a given region server, edit the _conf/hadoop-metrics2-h
 Restart the region server for the changes to take effect.
 
 To change the sampling rate for the default sink, edit the line beginning with `*.period`.
-To filter which metrics are emitted or to extend the metrics framework, see link:http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html
+To filter which metrics are emitted or to extend the metrics framework, see http://hadoop.apache.org/docs/current/api/org/apache/hadoop/metrics2/package-summary.html
 
 .HBase Metrics and Ganglia
 [NOTE]
@@ -1022,15 +1044,15 @@ Rather than listing each metric which HBase emits by default, you can browse thr
 Different metrics are exposed for the Master process and each region server process.
 
 .Procedure: Access a JSON Output of Available Metrics
-. After starting HBase, access the region server's web UI, at `http://REGIONSERVER_HOSTNAME:60030` by default (or port 16030 in HBase 1.0+).
+. After starting HBase, access the region server's web UI, at pass:[http://REGIONSERVER_HOSTNAME:60030] by default (or port 16030 in HBase 1.0+).
 . Click the [label]#Metrics Dump# link near the top.
   The metrics for the region server are presented as a dump of the JMX bean in JSON format.
   This will dump out all metrics names and their values.
-  To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes `http://REGIONSERVER_HOSTNAME:60030/jmx?description=true`.
+  To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes pass:[http://REGIONSERVER_HOSTNAME:60030/jmx?description=true].
   Not all beans and attributes have descriptions.
-. To view metrics for the Master, connect to the Master's web UI instead (defaults to `http://localhost:60010` or port 16010 in HBase 1.0+) and click its [label]#Metrics
+. To view metrics for the Master, connect to the Master's web UI instead (defaults to pass:[http://localhost:60010] or port 16010 in HBase 1.0+) and click its [label]#Metrics
   Dump# link.
-  To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes `http://REGIONSERVER_HOSTNAME:60010/jmx?description=true`.
+  To include metrics descriptions in the listing -- this can be useful when you are exploring what is available -- add a query string of `?description=true` so your URL becomes pass:[http://REGIONSERVER_HOSTNAME:60010/jmx?description=true].
   Not all beans and attributes have descriptions.
 
 
@@ -1264,7 +1286,8 @@ Have a look in the Web UI.
 
 == Cluster Replication
 
-NOTE: This information was previously available at link:http://hbase.apache.org/replication.html[Cluster Replication].
+NOTE: This information was previously available at
+link:http://hbase.apache.org#replication[Cluster Replication].
 
 HBase provides a cluster replication mechanism which allows you to keep one cluster's state synchronized with that of another cluster, using the write-ahead log (WAL) of the source cluster to propagate the changes.
 Some use cases for cluster replication include:
@@ -1297,6 +1320,14 @@ The master cluster relies on randomization to attempt to balance the stream of r
 It is expected that the slave cluster has storage capacity to hold the replicated data, as well as any data it is responsible for ingesting.
 If a slave cluster does run out of room, or is inaccessible for other reasons, it throws an error and the master retains the WAL and retries the replication at intervals.
 
+.Consistency Across Replicated Clusters
+[WARNING]
+====
+How your application builds on top of the HBase API matters when replication is in play. HBase's replication system provides at-least-once delivery of client edits for an enabled column family to each configured destination cluster. In the event of failure to reach a given destination, the replication system will retry sending edits in a way that might repeat a given message. Further more, there is not a guaranteed order of delivery for client edits. In the event of a RegionServer failing, recovery of the replication queue happens independent of recovery of the individual regions that server was previously handling. This means that it is possible for the not-yet-replicated edits to be serviced by a RegionServer that is currently slower to replicate than the one that handles edits from after the failure.
+
+The combination of these two properties (at-least-once delivery and the lack of message ordering) means that some destination clusters may end up in a different state if your application makes use of operations that are not idempotent, e.g. Increments.
+====
+
 .Terminology Changes
 [NOTE]
 ====
@@ -1344,13 +1375,16 @@ list_peers:: list all replication relationships known by this cluster
 enable_peer <ID>::
   Enable a previously-disabled replication relationship
 disable_peer <ID>::
-  Disable a replication relationship. HBase will no longer send edits to that peer cluster, but it still keeps track of all the new WALs that it will need to replicate if and when it is re-enabled. 
+  Disable a replication relationship. HBase will no longer send edits to that
+  peer cluster, but it still keeps track of all the new WALs that it will need
+  to replicate if and when it is re-enabled. WALs are retained when enabling or disabling
+  replication as long as peers exist.
 remove_peer <ID>::
   Disable and remove a replication relationship. HBase will no longer send edits to that peer cluster or keep track of WALs.
 enable_table_replication <TABLE_NAME>::
-  Enable the table replication switch for all it's column families. If the table is not found in the destination cluster then it will create one with the same name and column families. 
+  Enable the table replication switch for all its column families. If the table is not found in the destination cluster then it will create one with the same name and column families.
 disable_table_replication <TABLE_NAME>::
-  Disable the table replication switch for all it's column families. 
+  Disable the table replication switch for all its column families.
 
 === Verifying Replicated Data
 
@@ -1358,7 +1392,7 @@ The `VerifyReplication` MapReduce job, which is included in HBase, performs a sy
 +
 [source,bash]
 ----
-$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` "${HADOOP_HOME}/bin/hadoop" jar "${HBASE_HOME}/hbase-server-VERSION.jar" verifyrep --starttime=<timestamp> --stoptime=<timestamp> --families=<myFam> <ID> <tableName>
+$ HADOOP_CLASSPATH=`${HBASE_HOME}/bin/hbase classpath` "${HADOOP_HOME}/bin/hadoop" jar "${HBASE_HOME}/hbase-server-VERSION.jar" verifyrep --starttime=<timestamp> --endtime=<timestamp> --families=<myFam> <ID> <tableName>
 ----
 +
 The `VerifyReplication` command prints out `GOODROWS` and `BADROWS` counters to indicate rows that did and did not replicate correctly.
@@ -1404,6 +1438,7 @@ The following configuration settings are recommended for maintaining an even dis
 * Set `replication.source.sleepforretries` to `1` (1 second). This value, combined with the value of `replication.source.maxretriesmultiplier`, causes the retry cycle to last about 5 minutes.
 * Set `replication.sleep.before.failover` to `30000` (30 seconds) in the source cluster site configuration.
 
+[[cluster.replication.preserving.tags]]
 .Preserving Tags During Replication
 By default, the codec used for replication between clusters strips tags, such as cell-level ACLs, from cells.
 To prevent the tags from being stripped, you can use a different codec which does not strip them.
@@ -1469,7 +1504,7 @@ Speed is also limited by total size of the list of edits to replicate per slave,
 With this configuration, a master cluster region server with three slaves would use at most 192 MB to store data to replicate.
 This does not account for the data which was filtered but not garbage collected.
 
-Once the maximum size of edits has been buffered or the reader reaces the end of the WAL, the source thread stops reading and chooses at random a sink to replicate to (from the list that was generated by keeping only a subset of slave region servers). It directly issues a RPC to the chosen region server and waits for the method to return.
+Once the maximum size of edits has been buffered or the reader reaches the end of the WAL, the source thread stops reading and chooses at random a sink to replicate to (from the list that was generated by keeping only a subset of slave region servers). It directly issues a RPC to the chosen region server and waits for the method to return.
 If the RPC was successful, the source determines whether the current file has been emptied or it contains more data which needs to be read.
 If the file has been emptied, the source deletes the znode in the queue.
 Otherwise, it registers the new offset in the log's znode.
@@ -1483,6 +1518,8 @@ The default behavior is augmented so that if a log is past its TTL, the cleaning
 If the log is not found in any queues, the log will be deleted.
 The next time the cleaning process needs to look for a log, it starts by using its cached list.
 
+NOTE: WALs are saved when replication is enabled or disabled as long as peers exist.
+
 [[rs.failover.details]]
 ==== Region Server Failover
 
@@ -1641,12 +1678,7 @@ The following metrics are exposed at the global region server level and at the p
 | The name of the rs znode
 | rs
 
-| hbase.replication
-| Whether replication is enabled or disabled on a given
-                cluster
-| false
-
-| eplication.sleep.before.failover
+| replication.sleep.before.failover
 | How many milliseconds a worker should sleep before attempting to replicate
                 a dead region server's WAL queues.
 |
@@ -1664,6 +1696,197 @@ You can use the HBase Shell command `status 'replication'` to monitor the replic
 * `status 'replication', 'source'` -- prints the status for each replication source, sorted by hostname.
 * `status 'replication', 'sink'` -- prints the status for each replication sink, sorted by hostname.
 
+== Running Multiple Workloads On a Single Cluster
+
+HBase provides the following mechanisms for managing the performance of a cluster
+handling multiple workloads:
+. <<quota>>
+. <<request_queues>>
+. <<multiple-typed-queues>>
+
+[[quota]]
+=== Quotas
+HBASE-11598 introduces quotas, which allow you to throttle requests based on
+the following limits:
+
+. <<request-quotas,The number or size of requests(read, write, or read+write) in a given timeframe>>
+. <<namespace_quotas,The number of tables allowed in a namespace>>
+
+These limits can be enforced for a specified user, table, or namespace.
+
+.Enabling Quotas
+
+Quotas are disabled by default. To enable the feature, set the `hbase.quota.enabled`
+property to `true` in _hbase-site.xml_ file for all cluster nodes.
+
+.General Quota Syntax
+. THROTTLE_TYPE can be expressed as READ, WRITE, or the default type(read + write).
+. Timeframes  can be expressed in the following units: `sec`, `min`, `hour`, `day`
+. Request sizes can be expressed in the following units: `B` (bytes), `K` (kilobytes),
+`M` (megabytes), `G` (gigabytes), `T` (terabytes), `P` (petabytes)
+. Numbers of requests are expressed as an integer followed by the string `req`
+. Limits relating to time are expressed as req/time or size/time. For instance `10req/day`
+or `100P/hour`.
+. Numbers of tables or regions are expressed as integers.
+
+[[request-quotas]]
+.Setting Request Quotas
+You can set quota rules ahead of time, or you can change the throttle at runtime. The change
+will propagate after the quota refresh period has expired. This expiration period
+defaults to 5 minutes. To change it, modify the `hbase.quota.refresh.period` property
+in `hbase-site.xml`. This property is expressed in milliseconds and defaults to `300000`.
+
+----
+# Limit user u1 to 10 requests per second
+hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10req/sec'
+
+# Limit user u1 to 10 read requests per second
+hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => READ, USER => 'u1', LIMIT => '10req/sec'
+
+# Limit user u1 to 10 M per day everywhere
+hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => '10M/day'
+
+# Limit user u1 to 10 M write size per sec
+hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => WRITE, USER => 'u1', LIMIT => '10M/sec'
+
+# Limit user u1 to 5k per minute on table t2
+hbase> set_quota TYPE => THROTTLE, USER => 'u1', TABLE => 't2', LIMIT => '5K/min'
+
+# Limit user u1 to 10 read requests per sec on table t2
+hbase> set_quota TYPE => THROTTLE, THROTTLE_TYPE => READ, USER => 'u1', TABLE => 't2', LIMIT => '10req/sec'
+
+# Remove an existing limit from user u1 on namespace ns2
+hbase> set_quota TYPE => THROTTLE, USER => 'u1', NAMESPACE => 'ns2', LIMIT => NONE
+
+# Limit all users to 10 requests per hour on namespace ns1
+hbase> set_quota TYPE => THROTTLE, NAMESPACE => 'ns1', LIMIT => '10req/hour'
+
+# Limit all users to 10 T per hour on table t1
+hbase> set_quota TYPE => THROTTLE, TABLE => 't1', LIMIT => '10T/hour'
+
+# Remove all existing limits from user u1
+hbase> set_quota TYPE => THROTTLE, USER => 'u1', LIMIT => NONE
+
+# List all quotas for user u1 in namespace ns2
+hbase> list_quotas USER => 'u1, NAMESPACE => 'ns2'
+
+# List all quotas for namespace ns2
+hbase> list_quotas NAMESPACE => 'ns2'
+
+# List all quotas for table t1
+hbase> list_quotas TABLE => 't1'
+
+# list all quotas
+hbase> list_quotas
+----
+
+You can also place a global limit and exclude a user or a table from the limit by applying the
+`GLOBAL_BYPASS` property.
+----
+hbase> set_quota NAMESPACE => 'ns1', LIMIT => '100req/min'               # a per-namespace request limit
+hbase> set_quota USER => 'u1', GLOBAL_BYPASS => true                     # user u1 is not affected by the limit
+----
+
+[[namespace_quotas]]
+.Setting Namespace Quotas
+
+You can specify the maximum number of tables or regions allowed in a given namespace, either
+when you create the namespace or by altering an existing namespace, by setting the
+`hbase.namespace.quota.maxtables property`  on the namespace.
+
+.Limiting Tables Per Namespace
+----
+# Create a namespace with a max of 5 tables
+hbase> create_namespace 'ns1', {'hbase.namespace.quota.maxtables'=>'5'}
+
+# Alter an existing namespace to have a max of 8 tables
+hbase> alter_namespace 'ns2', {METHOD => 'set', 'hbase.namespace.quota.maxtables'=>'8'}
+
+# Show quota information for a namespace
+hbase> describe_namespace 'ns2'
+
+# Alter an existing namespace to remove a quota
+hbase> alter_namespace 'ns2', {METHOD => 'unset', NAME=>'hbase.namespace.quota.maxtables'}
+----
+
+.Limiting Regions Per Namespace
+----
+# Create a namespace with a max of 10 regions
+hbase> create_namespace 'ns1', {'hbase.namespace.quota.maxregions'=>'10'
+
+# Show quota information for a namespace
+hbase> describe_namespace 'ns1'
+
+# Alter an existing namespace to have a max of 20 tables
+hbase> alter_namespace 'ns2', {METHOD => 'set', 'hbase.namespace.quota.maxregions'=>'20'}
+
+# Alter an existing namespace to remove a quota
+hbase> alter_namespace 'ns2', {METHOD => 'unset', NAME=> 'hbase.namespace.quota.maxregions'}
+----
+
+[[request_queues]]
+=== Request Queues
+If no throttling policy is configured, when the RegionServer receives multiple requests,
+they are now placed into a queue waiting for a free execution slot (HBASE-6721).
+The simplest queue is a FIFO queue, where each request waits for all previous requests in the queue
+to finish before running. Fast or interactive queries can get stuck behind large requests.
+
+If you are able to guess how long a request will take, you can reorder requests by
+pushing the long requests to the end of the queue and allowing short requests to preempt
+them. Eventually, you must still execute the large requests and prioritize the new
+requests behind them. The short requests will be newer, so the result is not terrible,
+but still suboptimal compared to a mechanism which allows large requests to be split
+into multiple smaller ones.
+
+HBASE-10993 introduces such a system for deprioritizing long-running scanners. There
+are two types of queues, `fifo` and `deadline`. To configure the type of queue used,
+configure the `hbase.ipc.server.callqueue.type` property in `hbase-site.xml`. There
+is no way to estimate how long each request may take, so de-prioritization only affects
+scans, and is based on the number of \u201cnext\u201d calls a scan request has made. An assumption
+is made that when you are doing a full table scan, your job is not likely to be interactive,
+so if there are concurrent requests, you can delay long-running scans up to a limit tunable by
+setting the `hbase.ipc.server.queue.max.call.delay` property. The slope of the delay is calculated
+by a simple square root of `(numNextCall * weight)` where the weight is
+configurable by setting the `hbase.ipc.server.scan.vtime.weight` property.
+
+[[multiple-typed-queues]]
+=== Multiple-Typed Queues
+
+You can also prioritize or deprioritize different kinds of requests by configuring
+a specified number of dedicated handlers and queues. You can segregate the scan requests
+in a single queue with a single handler, and all the other available queues can service
+short `Get` requests.
+
+You can adjust the IPC queues and handlers based on the type of workload, using static
+tuning options. This approach is an interim first step that will eventually allow
+you to change the settings at runtime, and to dynamically adjust values based on the load.
+
+.Multiple Queues
+
+To avoid contention and separate different kinds of requests, configure the
+`hbase.ipc.server.callqueue.handler.factor` property, which allows you to increase the number of
+queues and control how many handlers can share the same queue., allows admins to increase the number
+of queues and decide how many handlers share the same queue.
+
+Using more queues reduces contention when adding a task to a queue or selecting it
+from a queue. You can even configure one queue per handler. The trade-off is that
+if some queues contain long-running tasks, a handler may need to wait to execute from that queue
+rather than stealing from another queue which has waiting tasks.
+
+.Read and Write Queues
+With multiple queues, you can now divide read and write requests, giving more priority
+(more queues) to one or the other type. Use the `hbase.ipc.server.callqueue.read.ratio`
+property to choose to serve more reads or more writes.
+
+.Get and Scan Queues
+Similar to the read/write split, you can split gets and scans by tuning the `hbase.ipc.server.callqueue.scan.ratio`
+property to give more priority to gets or to scans. A scan ratio of `0.1` will give
+more queue/handlers to the incoming gets, which means that more gets can be processed
+at the same time and that fewer scans can be executed at the same time. A value of
+`0.9` will give more queue/handlers to scans, so the number of scans executed will
+increase and the number of gets will decrease.
+
+
 [[ops.backup]]
 == HBase Backup
 
@@ -1704,12 +1927,12 @@ The act of copying these files creates new HDFS metadata, which is why a restore
 === Live Cluster Backup - Replication
 
 This approach assumes that there is a second cluster.
-See the HBase page on link:http://hbase.apache.org/replication.html[replication] for more information.
+See the HBase page on link:http://hbase.apache.org/book.html#replication[replication] for more information.
 
 [[ops.backup.live.copytable]]
 === Live Cluster Backup - CopyTable
 
-The <<copytable,copytable>> utility could either be used to copy data from one table to another on the same cluster, or to copy data to another table on another cluster.
+The <<copy.table,copytable>> utility could either be used to copy data from one table to another on the same cluster, or to copy data to another table on another cluster.
 
 Since the cluster is up, there is a risk that edits could be missed in the copy process.
 
@@ -1852,6 +2075,72 @@ The following example limits the above example to 200 MB/sec.
 $ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 -bandwidth 200
 ----
 
+[[snapshots_s3]]
+=== Storing Snapshots in an Amazon S3 Bucket
+
+You can store and retrieve snapshots from Amazon S3, using the following procedure.
+
+NOTE: You can also store snapshots in Microsoft Azure Blob Storage. See <<snapshots_azure>>.
+
+.Prerequisites
+- You must be using HBase 1.0 or higher and Hadoop 2.6.1 or higher, which is the first
+configuration that uses the Amazon AWS SDK.
+- You must use the `s3a://` protocol to connect to Amazon S3. The older `s3n://`
+and `s3://` protocols have various limitations and do not use the Amazon AWS SDK.
+- The `s3a://` URI must be configured and available on the server where you run
+the commands to export and restore the snapshot.
+
+After you have fulfilled the prerequisites, take the snapshot like you normally would.
+Afterward, you can export it using the `org.apache.hadoop.hbase.snapshot.ExportSnapshot`
+command like the one below, substituting your own `s3a://` path in the `copy-from`
+or `copy-to` directive and substituting or modifying other options as required:
+
+----
+$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
+    -snapshot MySnapshot \
+    -copy-from hdfs://srv2:8082/hbase \
+    -copy-to s3a://<bucket>/<namespace>/hbase \
+    -chuser MyUser \
+    -chgroup MyGroup \
+    -chmod 700 \
+    -mappers 16
+----
+
+----
+$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \
+    -snapshot MySnapshot
+    -copy-from s3a://<bucket>/<namespace>/hbase \
+    -copy-to hdfs://srv2:8082/hbase \
+    -chuser MyUser \
+    -chgroup MyGroup \
+    -chmod 700 \
+    -mappers 16
+----
+
+You can also use the `org.apache.hadoop.hbase.snapshot.SnapshotInfo` utility with the `s3a://` path by including the
+`-remote-dir` option.
+
+----
+$ hbase org.apache.hadoop.hbase.snapshot.SnapshotInfo \
+    -remote-dir s3a://<bucket>/<namespace>/hbase \
+    -list-snapshots
+----
+
+[[snapshots_azure]]
+== Storing Snapshots in Microsoft Azure Blob Storage
+
+You can store snapshots in Microsoft Azure Blog Storage using the same techniques
+as in <<snapshots_s3>>.
+
+.Prerequisites
+- You must be using HBase 1.2 or higher with Hadoop 2.7.1 or
+  higher. No version of HBase supports Hadoop 2.7.0.
+- Your hosts must be configured to be aware of the Azure blob storage filesystem.
+  See http://hadoop.apache.org/docs/r2.7.1/hadoop-azure/index.html.
+
+After you meet the prerequisites, follow the instructions
+in <<snapshots_s3>>, replacingthe protocol specifier with `wasb://` or `wasbs://`.
+
 [[ops.capacity]]
 == Capacity Planning and Region Sizing
 
@@ -1887,7 +2176,7 @@ Aside from the disk space necessary to store the data, one RS may not be able to
 [[ops.capacity.nodes.throughput]]
 ==== Read/Write throughput
 
-Number of nodes can also be driven by required thoughput for reads and/or writes.
+Number of nodes can also be driven by required throughput for reads and/or writes.
 The throughput one can get per node depends a lot on data (esp.
 key/value sizes) and request patterns, as well as node and system configuration.
 Planning should be done for peak load if it is likely that the load would be the main driver of the increase of the node count.
@@ -2012,7 +2301,7 @@ See <<compaction,compaction>> for some details.
 
 When provisioning for large data sizes, however, it's good to keep in mind that compactions can affect write throughput.
 Thus, for write-intensive workloads, you may opt for less frequent compactions and more store files per regions.
-Minimum number of files for compactions (`hbase.hstore.compaction.min`) can be set to higher value; <<hbase.hstore.blockingstorefiles,hbase.hstore.blockingStoreFiles>> should also be increased, as more files might accumulate in such case.
+Minimum number of files for compactions (`hbase.hstore.compaction.min`) can be set to higher value; <<hbase.hstore.blockingStoreFiles,hbase.hstore.blockingStoreFiles>> should also be increased, as more files might accumulate in such case.
 You may also consider manually managing compactions: <<managed.compactions,managed.compactions>>
 
 [[ops.capacity.config.presplit]]
@@ -2052,7 +2341,7 @@ or in code it would be as follows:
 
 [source,java]
 ----
-void rename(Admin admin, String oldTableName, String newTableName) {
+void rename(Admin admin, String oldTableName, TableName newTableName) {
   String snapshotName = randomName();
   admin.disableTable(oldTableName);
   admin.snapshot(snapshotName, oldTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/other_info.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/other_info.adoc b/src/main/asciidoc/_chapters/other_info.adoc
index 046b747..8bcbe0f 100644
--- a/src/main/asciidoc/_chapters/other_info.adoc
+++ b/src/main/asciidoc/_chapters/other_info.adoc
@@ -31,50 +31,50 @@
 [[other.info.videos]]
 === HBase Videos
 
-.Introduction to HBase 
-* link:http://www.cloudera.com/content/cloudera/en/resources/library/presentation/chicago_data_summit_apache_hbase_an_introduction_todd_lipcon.html[Introduction to HBase] by Todd Lipcon (Chicago Data Summit 2011). 
-* link:http://www.cloudera.com/videos/intorduction-hbase-todd-lipcon[Introduction to HBase] by Todd Lipcon (2010).         
-link:http://www.cloudera.com/videos/hadoop-world-2011-presentation-video-building-realtime-big-data-services-at-facebook-with-hadoop-and-hbase[Building Real Time Services at Facebook with HBase] by Jonathan Gray (Hadoop World 2011). 
+.Introduction to HBase
+* link:http://www.cloudera.com/content/cloudera/en/resources/library/presentation/chicago_data_summit_apache_hbase_an_introduction_todd_lipcon.html[Introduction to HBase] by Todd Lipcon (Chicago Data Summit 2011).
+* link:http://www.cloudera.com/videos/intorduction-hbase-todd-lipcon[Introduction to HBase] by Todd Lipcon (2010).
+link:http://www.cloudera.com/videos/hadoop-world-2011-presentation-video-building-realtime-big-data-services-at-facebook-with-hadoop-and-hbase[Building Real Time Services at Facebook with HBase] by Jonathan Gray (Hadoop World 2011).
 
-link:http://www.cloudera.com/videos/hw10_video_how_stumbleupon_built_and_advertising_platform_using_hbase_and_hadoop[HBase and Hadoop, Mixing Real-Time and Batch Processing at StumbleUpon] by JD Cryans (Hadoop World 2010). 
+link:http://www.cloudera.com/videos/hw10_video_how_stumbleupon_built_and_advertising_platform_using_hbase_and_hadoop[HBase and Hadoop, Mixing Real-Time and Batch Processing at StumbleUpon] by JD Cryans (Hadoop World 2010).
 
 [[other.info.pres]]
 === HBase Presentations (Slides)
 
-link:http://www.cloudera.com/content/cloudera/en/resources/library/hadoopworld/hadoop-world-2011-presentation-video-advanced-hbase-schema-design.html[Advanced HBase Schema Design] by Lars George (Hadoop World 2011). 
+link:http://www.cloudera.com/content/cloudera/en/resources/library/hadoopworld/hadoop-world-2011-presentation-video-advanced-hbase-schema-design.html[Advanced HBase Schema Design] by Lars George (Hadoop World 2011).
 
-link:http://www.slideshare.net/cloudera/chicago-data-summit-apache-hbase-an-introduction[Introduction to HBase] by Todd Lipcon (Chicago Data Summit 2011). 
+link:http://www.slideshare.net/cloudera/chicago-data-summit-apache-hbase-an-introduction[Introduction to HBase] by Todd Lipcon (Chicago Data Summit 2011).
 
-link:http://www.slideshare.net/cloudera/hw09-practical-h-base-getting-the-most-from-your-h-base-install[Getting The Most From Your HBase Install] by Ryan Rawson, Jonathan Gray (Hadoop World 2009). 
+link:http://www.slideshare.net/cloudera/hw09-practical-h-base-getting-the-most-from-your-h-base-install[Getting The Most From Your HBase Install] by Ryan Rawson, Jonathan Gray (Hadoop World 2009).
 
 [[other.info.papers]]
 === HBase Papers
 
-link:http://research.google.com/archive/bigtable.html[BigTable] by Google (2006). 
+link:http://research.google.com/archive/bigtable.html[BigTable] by Google (2006).
 
-link:http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html[HBase and HDFS Locality] by Lars George (2010). 
+link:http://www.larsgeorge.com/2010/05/hbase-file-locality-in-hdfs.html[HBase and HDFS Locality] by Lars George (2010).
 
-link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation: The Mixed Blessings of Non-Relational Databases] by Ian Varley (2009). 
+link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation: The Mixed Blessings of Non-Relational Databases] by Ian Varley (2009).
 
 [[other.info.sites]]
 === HBase Sites
 
-link:http://www.cloudera.com/blog/category/hbase/[Cloudera's HBase Blog] has a lot of links to useful HBase information. 
+link:https://blog.cloudera.com/blog/category/hbase/[Cloudera's HBase Blog] has a lot of links to useful HBase information.
 
-* link:http://www.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/[CAP Confusion] is a relevant entry for background information on distributed storage systems.        
+* link:https://blog.cloudera.com/blog/2010/04/cap-confusion-problems-with-partition-tolerance/[CAP Confusion] is a relevant entry for background information on distributed storage systems.
 
-link:http://wiki.apache.org/hadoop/HBase/HBasePresentations[HBase Wiki] has a page with a number of presentations. 
+link:http://wiki.apache.org/hadoop/HBase/HBasePresentations[HBase Wiki] has a page with a number of presentations.
 
-link:http://refcardz.dzone.com/refcardz/hbase[HBase RefCard] from DZone. 
+link:http://refcardz.dzone.com/refcardz/hbase[HBase RefCard] from DZone.
 
 [[other.info.books]]
 === HBase Books
 
-link:http://shop.oreilly.com/product/0636920014348.do[HBase:  The Definitive Guide] by Lars George. 
+link:http://shop.oreilly.com/product/0636920014348.do[HBase:  The Definitive Guide] by Lars George.
 
 [[other.info.books.hadoop]]
 === Hadoop Books
 
-link:http://shop.oreilly.com/product/9780596521981.do[Hadoop:  The Definitive Guide] by Tom White. 
+link:http://shop.oreilly.com/product/9780596521981.do[Hadoop:  The Definitive Guide] by Tom White.
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/performance.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/performance.adoc b/src/main/asciidoc/_chapters/performance.adoc
index 526fd01..5f27640 100644
--- a/src/main/asciidoc/_chapters/performance.adoc
+++ b/src/main/asciidoc/_chapters/performance.adoc
@@ -48,6 +48,11 @@ Use a 64-bit platform (and 64-bit JVM).
 Watch out for swapping.
 Set `swappiness` to 0.
 
+[[perf.os.cpu]]
+=== CPU
+Make sure you have set up your Hadoop to use native, hardware checksumming.
+See link:[hadoop.native.lib].
+
 [[perf.network]]
 == Network
 
@@ -88,7 +93,7 @@ Multiple rack configurations carry the same potential issues as multiple switche
 * Poor switch capacity performance
 * Insufficient uplink to another rack
 
-If the the switches in your rack have appropriate switching capacity to handle all the hosts at full speed, the next most likely issue will be caused by homing more of your cluster across racks.
+If the switches in your rack have appropriate switching capacity to handle all the hosts at full speed, the next most likely issue will be caused by homing more of your cluster across racks.
 The easiest way to avoid issues when spanning multiple racks is to use port trunking to create a bonded uplink to other racks.
 The downside of this method however, is in the overhead of ports that could potentially be used.
 An example of this is, creating an 8Gbps port channel from rack A to rack B, using 8 of your 24 ports to communicate between racks gives you a poor ROI, using too few however can mean you're not getting the most out of your cluster.
@@ -102,14 +107,14 @@ Are all the network interfaces functioning correctly? Are you sure? See the Trou
 
 [[perf.network.call_me_maybe]]
 === Network Consistency and Partition Tolerance
-The link:http://en.wikipedia.org/wiki/CAP_theorem[CAP Theorem] states that a distributed system can maintain two out of the following three charateristics: 
-- *C*onsistency -- all nodes see the same data. 
+The link:http://en.wikipedia.org/wiki/CAP_theorem[CAP Theorem] states that a distributed system can maintain two out of the following three characteristics:
+- *C*onsistency -- all nodes see the same data.
 - *A*vailability -- every request receives a response about whether it succeeded or failed.
 - *P*artition tolerance -- the system continues to operate even if some of its components become unavailable to the others.
 
-HBase favors consistency and partition tolerance, where a decision has to be made. Coda Hale explains why partition tolerance is so important, in http://codahale.com/you-cant-sacrifice-partition-tolerance/. 
+HBase favors consistency and partition tolerance, where a decision has to be made. Coda Hale explains why partition tolerance is so important, in http://codahale.com/you-cant-sacrifice-partition-tolerance/.
 
-Robert Yokota used an automated testing framework called link:https://aphyr.com/tags/jepsen[Jepson] to test HBase's partition tolerance in the face of network partitions, using techniques modeled after Aphyr's link:https://aphyr.com/posts/281-call-me-maybe-carly-rae-jepsen-and-the-perils-of-network-partitions[Call Me Maybe] series. The results, available as a link:http://old.eng.yammer.com/call-me-maybe-hbase/[blog post] and an link:http://old.eng.yammer.com/call-me-maybe-hbase-addendum/[addendum], show that HBase performs correctly.
+Robert Yokota used an automated testing framework called link:https://aphyr.com/tags/jepsen[Jepson] to test HBase's partition tolerance in the face of network partitions, using techniques modeled after Aphyr's link:https://aphyr.com/posts/281-call-me-maybe-carly-rae-jepsen-and-the-perils-of-network-partitions[Call Me Maybe] series. The results, available as a link:https://rayokota.wordpress.com/2015/09/30/call-me-maybe-hbase/[blog post] and an link:https://rayokota.wordpress.com/2015/09/30/call-me-maybe-hbase-addendum/[addendum], show that HBase performs correctly.
 
 [[jvm]]
 == Java
@@ -137,6 +142,9 @@ It describes configurations to lower the amount of young GC during write-heavy l
 If you do not have HBASE-8163 installed, and you are trying to improve your young GC times, one trick to consider -- courtesy of our Liang Xie -- is to set the GC config `-XX:PretenureSizeThreshold` in _hbase-env.sh_ to be just smaller than the size of `hbase.hregion.memstore.mslab.chunksize` so MSLAB allocations happen in the tenured space directly rather than first in the young gen.
 You'd do this because these MSLAB allocations are going to likely make it to the old gen anyways and rather than pay the price of a copies between s0 and s1 in eden space followed by the copy up from young to old gen after the MSLABs have achieved sufficient tenure, save a bit of YGC churn and allocate in the old gen directly.
 
+Other sources of long GCs can be the JVM itself logging.
+See link:https://engineering.linkedin.com/blog/2016/02/eliminating-large-jvm-gc-pauses-caused-by-background-io-traffic[Eliminating Large JVM GC Pauses Caused by Background IO Traffic]
+
 For more information about GC logs, see <<trouble.log.gc>>.
 
 Consider also enabling the off-heap Block Cache.
@@ -196,7 +204,13 @@ tableDesc.addFamily(cfDesc);
 ----
 ====
 
-See the API documentation for link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig].
+See the API documentation for
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig].
+
+To see prefetch in operation, enable TRACE level logging on
+`org.apache.hadoop.hbase.io.hfile.HFileReaderImpl` in hbase-2.0+
+or on `org.apache.hadoop.hbase.io.hfile.HFileReaderV2` in earlier versions, hbase-1.x, of HBase.
+
 
 [[perf.rs.memstore.size]]
 === `hbase.regionserver.global.memstore.size`
@@ -213,7 +227,7 @@ This memory setting is often adjusted for the RegionServer process depending on
 [[perf.hstore.blockingstorefiles]]
 === `hbase.hstore.blockingStoreFiles`
 
-See <<hbase.hstore.blockingstorefiles>>.
+See <<hbase.hstore.blockingStoreFiles>>.
 If there is blocking in the RegionServer logs, increasing this can help.
 
 [[perf.hregion.memstore.block.multiplier]]
@@ -324,7 +338,7 @@ HBase includes some tuning mechanisms for folding the Bloom filter to reduce the
 
 Bloom filters were introduced in link:https://issues.apache.org/jira/browse/HBASE-1200[HBASE-1200].
 Since HBase 0.96, row-based Bloom filters are enabled by default.
-(link:https://issues.apache.org/jira/browse/HBASE-8450[HBASE-])
+(link:https://issues.apache.org/jira/browse/HBASE-8450[HBASE-8450])
 
 For more information on Bloom filters in relation to HBase, see <<blooms>> for more information, or the following Quora discussion: link:http://www.quora.com/How-are-bloom-filters-used-in-HBase[How are bloom filters used in HBase?].
 
@@ -352,7 +366,7 @@ Bloom filters need to be rebuilt upon deletion, so may not be appropriate in env
 
 Bloom filters are enabled on a Column Family.
 You can do this by using the setBloomFilterType method of HColumnDescriptor or using the HBase API.
-Valid values are `NONE` (the default), `ROW`, or `ROWCOL`.
+Valid values are `NONE`, `ROW` (default), or `ROWCOL`.
 See <<bloom.filters.when>> for more information on `ROW` versus `ROWCOL`.
 See also the API documentation for link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HColumnDescriptor.html[HColumnDescriptor].
 
@@ -373,17 +387,17 @@ You can configure the following settings in the _hbase-site.xml_.
 | Default
 | Description
 
-| io.hfile.bloom.enabled
+| io.storefile.bloom.enabled
 | yes
 | Set to no to kill bloom filters server-wide if something goes wrong
 
-| io.hfile.bloom.error.rate
+| io.storefile.bloom.error.rate
 | .01
 | The average false positive rate for bloom filters. Folding is used to
                   maintain the false positive rate. Expressed as a decimal representation of a
                   percentage.
 
-| io.hfile.bloom.max.fold
+| io.storefile.bloom.max.fold
 | 7
 | The guaranteed maximum fold rate. Changing this setting should not be
                   necessary and is not recommended.
@@ -397,7 +411,7 @@ You can configure the following settings in the _hbase-site.xml_.
 | Master switch to enable Delete Family Bloom filters and store them in the StoreFile.
 
 | io.storefile.bloom.block.size
-| 65536
+| 131072
 | Target Bloom block size. Bloom filter blocks of approximately this size
                   are interleaved with data blocks.
 
@@ -485,7 +499,7 @@ For bulk imports, this means that all clients will write to the same region unti
 A useful pattern to speed up the bulk import process is to pre-create empty regions.
 Be somewhat conservative in this, because too-many regions can actually degrade performance.
 
-There are two different approaches to pre-creating splits.
+There are two different approaches to pre-creating splits using the HBase API.
 The first approach is to rely on the default `Admin` strategy (which is implemented in `Bytes.split`)...
 
 [source,java]
@@ -497,7 +511,7 @@ int numberOfRegions = ...;  // # of regions to create
 admin.createTable(table, startKey, endKey, numberOfRegions);
 ----
 
-And the other approach is to define the splits yourself...
+And the other approach, using the HBase API, is to define the splits yourself...
 
 [source,java]
 ----
@@ -505,8 +519,23 @@ byte[][] splits = ...;   // create your own splits
 admin.createTable(table, splits);
 ----
 
+You can achieve a similar effect using the HBase Shell to create tables by specifying split options. 
+
+[source]
+----
+# create table with specific split points
+hbase>create 't1','f1',SPLITS => ['\x10\x00', '\x20\x00', '\x30\x00', '\x40\x00']
+
+# create table with four regions based on random bytes keys
+hbase>create 't2','f1', { NUMREGIONS => 4 , SPLITALGO => 'UniformSplit' }
+
+# create table with five regions based on hex keys
+create 't3','f1', { NUMREGIONS => 5, SPLITALGO => 'HexStringSplit' }
+----
+
 See <<rowkey.regionsplits>> for issues related to understanding your keyspace and pre-creating regions.
 See <<manual_region_splitting_decisions,manual region splitting decisions>>  for discussion on manually pre-splitting regions.
+See <<tricks.pre-split>> for more details of using the HBase Shell to pre-split tables.
 
 [[def.log.flush]]
 ===  Table Creation: Deferred Log Flush
@@ -546,7 +575,7 @@ To disable the WAL, see <<wal.disable>>.
 === HBase Client: Group Puts by RegionServer
 
 In addition to using the writeBuffer, grouping `Put`s by RegionServer can reduce the number of client RPC calls per writeBuffer flush.
-There is a utility `HTableUtil` currently on TRUNK that does this, but you can either copy that or implement your own version for those still on 0.90.x or earlier.
+There is a utility `HTableUtil` currently on MASTER that does this, but you can either copy that or implement your own version for those still on 0.90.x or earlier.
 
 [[perf.hbase.write.mr.reducer]]
 === MapReduce: Skip The Reducer
@@ -555,7 +584,7 @@ When writing a lot of data to an HBase table from a MR job (e.g., with link:http
 When a Reducer step is used, all of the output (Puts) from the Mapper will get spooled to disk, then sorted/shuffled to other Reducers that will most likely be off-node.
 It's far more efficient to just write directly to HBase.
 
-For summary jobs where HBase is used as a source and a sink, then writes will be coming from the Reducer step (e.g., summarize values then write out result). This is a different processing problem than from the the above case.
+For summary jobs where HBase is used as a source and a sink, then writes will be coming from the Reducer step (e.g., summarize values then write out result). This is a different processing problem than from the above case.
 
 [[perf.one.region]]
 === Anti-Pattern: One Hot Region
@@ -564,7 +593,7 @@ If all your data is being written to one region at a time, then re-read the sect
 
 Also, if you are pre-splitting regions and all your data is _still_ winding up in a single region even though your keys aren't monotonically increasing, confirm that your keyspace actually works with the split strategy.
 There are a variety of reasons that regions may appear "well split" but won't work with your data.
-As the HBase client communicates directly with the RegionServers, this can be obtained via link:hhttp://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#getRegionLocation(byte[])[Table.getRegionLocation].
+As the HBase client communicates directly with the RegionServers, this can be obtained via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#getRegionLocation(byte%5B%5D)[Table.getRegionLocation].
 
 See <<precreate.regions>>, as well as <<perf.configurations>>
 
@@ -606,7 +635,7 @@ When columns are selected explicitly with `scan.addColumn`, HBase will schedule
 When rows have few columns and each column has only a few versions this can be inefficient.
 A seek operation is generally slower if does not seek at least past 5-10 columns/versions or 512-1024 bytes.
 
-In order to opportunistically look ahead a few columns/versions to see if the next column/version can be found that way before a seek operation is scheduled, a new attribute `Scan.HINT_LOOKAHEAD` can be set the on Scan object.
+In order to opportunistically look ahead a few columns/versions to see if the next column/version can be found that way before a seek operation is scheduled, a new attribute `Scan.HINT_LOOKAHEAD` can be set on the Scan object.
 The following code instructs the RegionServer to attempt two iterations of next before a seek is scheduled:
 
 [source,java]
@@ -676,7 +705,7 @@ Enabling Bloom Filters can save your having to go to disk and can help improve r
 link:http://en.wikipedia.org/wiki/Bloom_filter[Bloom filters] were developed over in link:https://issues.apache.org/jira/browse/HBASE-1200[HBase-1200 Add bloomfilters].
 For description of the development process -- why static blooms rather than dynamic -- and for an overview of the unique properties that pertain to blooms in HBase, as well as possible future directions, see the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters in HBase] attached to link:https://issues.apache.org/jira/browse/HBASE-1200[HBASE-1200].
 The bloom filters described here are actually version two of blooms in HBase.
-In versions up to 0.19.x, HBase had a dynamic bloom option based on work done by the link:http://www.one-lab.org[European Commission One-Lab Project 034819].
+In versions up to 0.19.x, HBase had a dynamic bloom option based on work done by the link:http://www.one-lab.org/[European Commission One-Lab Project 034819].
 The core of the HBase bloom work was later pulled up into Hadoop to implement org.apache.hadoop.io.BloomMapFile.
 Version 1 of HBase blooms never worked that well.
 Version 2 is a rewrite from scratch though again it starts with the one-lab work.
@@ -704,20 +733,20 @@ Stored in the LRU cache, if it is enabled (It's enabled by default).
 [[config.bloom]]
 ==== Bloom Filter Configuration
 
-===== `io.hfile.bloom.enabled` global kill switch
+===== `io.storefile.bloom.enabled` global kill switch
 
-`io.hfile.bloom.enabled` in `Configuration` serves as the kill switch in case something goes wrong.
+`io.storefile.bloom.enabled` in `Configuration` serves as the kill switch in case something goes wrong.
 Default = `true`.
 
-===== `io.hfile.bloom.error.rate`
+===== `io.storefile.bloom.error.rate`
 
-`io.hfile.bloom.error.rate` = average false positive rate.
+`io.storefile.bloom.error.rate` = average false positive rate.
 Default = 1%. Decrease rate by � (e.g.
 to .5%) == +1 bit per bloom entry.
 
-===== `io.hfile.bloom.max.fold`
+===== `io.storefile.bloom.max.fold`
 
-`io.hfile.bloom.max.fold` = guaranteed minimum fold rate.
+`io.storefile.bloom.max.fold` = guaranteed minimum fold rate.
 Most people should leave this alone.
 Default = 7, or can collapse to at least 1/128th of original size.
 See the _Development Process_ section of the document link:https://issues.apache.org/jira/secure/attachment/12444007/Bloom_Filters_in_HBase.pdf[BloomFilters in HBase] for more on what this option means.
@@ -730,7 +759,7 @@ However, if hedged reads are enabled, the client waits some configurable amount
 Whichever read returns first is used, and the other read request is discarded.
 Hedged reads can be helpful for times where a rare slow read is caused by a transient error such as a failing disk or flaky network connection.
 
-Because a HBase RegionServer is a HDFS client, you can enable hedged reads in HBase, by adding the following properties to the RegionServer's hbase-site.xml and tuning the values to suit your environment.
+Because an HBase RegionServer is a HDFS client, you can enable hedged reads in HBase, by adding the following properties to the RegionServer's hbase-site.xml and tuning the values to suit your environment.
 
 .Configuration for Hedged Reads
 * `dfs.client.hedged.read.threadpool.size` - the number of threads dedicated to servicing hedged reads.
@@ -781,7 +810,8 @@ Be aware that `Table.delete(Delete)` doesn't use the writeBuffer.
 It will execute an RegionServer RPC with each invocation.
 For a large number of deletes, consider `Table.delete(List)`.
 
-See http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete%28org.apache.hadoop.hbase.client.Delete%29
+See
++++<a href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#delete%28org.apache.hadoop.hbase.client.Delete%29">hbase.client.Delete</a>+++.
 
 [[perf.hdfs]]
 == HDFS
@@ -868,7 +898,7 @@ If you are running on EC2 and post performance questions on the dist-list, pleas
 == Collocating HBase and MapReduce
 
 It is often recommended to have different clusters for HBase and MapReduce.
-A better qualification of this is: don't collocate a HBase that serves live requests with a heavy MR workload.
+A better qualification of this is: don't collocate an HBase that serves live requests with a heavy MR workload.
 OLTP and OLAP-optimized systems have conflicting requirements and one will lose to the other, usually the former.
 For example, short latency-sensitive disk reads will have to wait in line behind longer reads that are trying to squeeze out as much throughput as possible.
 MR jobs that write to HBase will also generate flushes and compactions, which will in turn invalidate blocks in the <<block.cache>>.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/preface.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/preface.adoc b/src/main/asciidoc/_chapters/preface.adoc
index 960fcc4..7d244bd 100644
--- a/src/main/asciidoc/_chapters/preface.adoc
+++ b/src/main/asciidoc/_chapters/preface.adoc
@@ -29,20 +29,29 @@
 
 This is the official reference guide for the link:http://hbase.apache.org/[HBase] version it ships with.
 
-Herein you will find either the definitive documentation on an HBase topic as of its standing when the referenced HBase version shipped, or it will point to the location in link:http://hbase.apache.org/apidocs/index.html[Javadoc], link:https://issues.apache.org/jira/browse/HBASE[JIRA] or link:http://wiki.apache.org/hadoop/Hbase[wiki] where the pertinent information can be found.
+Herein you will find either the definitive documentation on an HBase topic as of its
+standing when the referenced HBase version shipped, or it will point to the location
+in link:http://hbase.apache.org/apidocs/index.html[Javadoc] or
+link:https://issues.apache.org/jira/browse/HBASE[JIRA] where the pertinent information can be found.
 
 .About This Guide
-This reference guide is a work in progress. The source for this guide can be found in the _src/main/asciidoc directory of the HBase source. This reference guide is marked up using link:http://asciidoc.org/[AsciiDoc] from which the finished guide is generated as part of the 'site' build target. Run
+This reference guide is a work in progress. The source for this guide can be found in the
+_src/main/asciidoc directory of the HBase source. This reference guide is marked up
+using link:http://asciidoc.org/[AsciiDoc] from which the finished guide is generated as part of the
+'site' build target. Run
 [source,bourne]
 ----
 mvn site
 ----
 to generate this documentation.
 Amendments and improvements to the documentation are welcomed.
-Click link:https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12310753&issuetype=1&components=12312132&summary=SHORT+DESCRIPTION[this link] to file a new documentation bug against Apache HBase with some values pre-selected.
+Click
+link:https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12310753&issuetype=1&components=12312132&summary=SHORT+DESCRIPTION[this link]
+to file a new documentation bug against Apache HBase with some values pre-selected.
 
 .Contributing to the Documentation
-For an overview of AsciiDoc and suggestions to get started contributing to the documentation, see the <<appendix_contributing_to_documentation,relevant section later in this documentation>>.
+For an overview of AsciiDoc and suggestions to get started contributing to the documentation,
+see the <<appendix_contributing_to_documentation,relevant section later in this documentation>>.
 
 .Heads-up if this is your first foray into the world of distributed computing...
 If this is your first foray into the wonderful world of Distributed Computing, then you are in for some interesting times.
@@ -57,8 +66,43 @@ Yours, the HBase Community.
 
 .Reporting Bugs
 
-Please use link:https://issues.apache.org/jira/browse/hbase[JIRA] to report non-security-related bugs. 
+Please use link:https://issues.apache.org/jira/browse/hbase[JIRA] to report non-security-related bugs.
 
 To protect existing HBase installations from new vulnerabilities, please *do not* use JIRA to report security-related bugs. Instead, send your report to the mailing list private@apache.org, which allows anyone to send messages, but restricts who can read them. Someone on that list will contact you to follow up on your report.
 
+[[hbase_supported_tested_definitions]]
+.Support and Testing Expectations
+
+The phrases /supported/, /not supported/, /tested/, and /not tested/ occur several
+places throughout this guide. In the interest of clarity, here is a brief explanation
+of what is generally meant by these phrases, in the context of HBase.
+
+NOTE: Commercial technical support for Apache HBase is provided by many Hadoop vendors.
+This is not the sense in which the term /support/ is used in the context of the
+Apache HBase project. The Apache HBase team assumes no responsibility for your
+HBase clusters, your configuration, or your data.
+
+Supported::
+  In the context of Apache HBase, /supported/ means that HBase is designed to work
+  in the way described, and deviation from the defined behavior or functionality should
+  be reported as a bug.
+
+Not Supported::
+  In the context of Apache HBase, /not supported/ means that a use case or use pattern
+  is not expected to work and should be considered an antipattern. If you think this
+  designation should be reconsidered for a given feature or use pattern, file a JIRA
+  or start a discussion on one of the mailing lists.
+
+Tested::
+  In the context of Apache HBase, /tested/ means that a feature is covered by unit
+  or integration tests, and has been proven to work as expected.
+
+Not Tested::
+  In the context of Apache HBase, /not tested/ means that a feature or use pattern
+  may or may notwork in a given way, and may or may not corrupt your data or cause
+  operational issues. It is an unknown, and there are no guarantees. If you can provide
+  proof that a feature designated as /not tested/ does work in a given way, please
+  submit the tests and/or the metrics so that other users can gain certainty about
+  such features or use patterns.
+
 :numbered:


[5/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/developer.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
index c3ba0a2..f064cb6 100644
--- a/src/main/asciidoc/_chapters/developer.adoc
+++ b/src/main/asciidoc/_chapters/developer.adoc
@@ -40,14 +40,14 @@ See link:http://search-hadoop.com/m/DHED43re96[What label
 
 Before you get started submitting code to HBase, please refer to <<developing,developing>>.
 
-As Apache HBase is an Apache Software Foundation project, see <<asf,asf>>            for more information about how the ASF functions. 
+As Apache HBase is an Apache Software Foundation project, see <<asf,asf>>            for more information about how the ASF functions.
 
 [[mailing.list]]
 === Mailing Lists
 
 Sign up for the dev-list and the user-list.
 See the link:http://hbase.apache.org/mail-lists.html[mailing lists] page.
-Posing questions - and helping to answer other people's questions - is encouraged! There are varying levels of experience on both lists so patience and politeness are encouraged (and please stay on topic.) 
+Posing questions - and helping to answer other people's questions - is encouraged! There are varying levels of experience on both lists so patience and politeness are encouraged (and please stay on topic.)
 
 [[irc]]
 === Internet Relay Chat (IRC)
@@ -58,7 +58,7 @@ FreeNode offers a web-based client, but most people prefer a native client, and
 === Jira
 
 Check for existing issues in link:https://issues.apache.org/jira/browse/HBASE[Jira].
-If it's either a new feature request, enhancement, or a bug, file a ticket. 
+If it's either a new feature request, enhancement, or a bug, file a ticket.
 
 To check for existing issues which you can tackle as a beginner, search for link:https://issues.apache.org/jira/issues/?jql=project%20%3D%20HBASE%20AND%20labels%20in%20(beginner)[issues in JIRA tagged with the label 'beginner'].
 
@@ -89,11 +89,12 @@ GIT is our repository of record for all but the Apache HBase website.
 We used to be on SVN.
 We migrated.
 See link:https://issues.apache.org/jira/browse/INFRA-7768[Migrate Apache HBase SVN Repos to Git].
-Updating hbase.apache.org still requires use of SVN (See <<hbase.org,hbase.org>>). See link:http://hbase.apache.org/source-repository.html[Source Code
-                Management] page for contributor and committer links or seach for HBase on the link:http://git.apache.org/[Apache Git] page.
+See link:http://hbase.apache.org/source-repository.html[Source Code
+                Management] page for contributor and committer links or search for HBase on the link:http://git.apache.org/[Apache Git] page.
 
 == IDEs
 
+[[eclipse]]
 === Eclipse
 
 [[eclipse.code.formatting]]
@@ -104,10 +105,10 @@ We encourage you to have this formatter in place in eclipse when editing HBase c
 
 .Procedure: Load the HBase Formatter Into Eclipse
 . Open the  menu item.
-. In Preferences, click the  menu item.
+. In Preferences, Go to `Java->Code Style->Formatter`.
 . Click btn:[Import] and browse to the location of the _hbase_eclipse_formatter.xml_ file, which is in the _dev-support/_ directory.
   Click btn:[Apply].
-. Still in Preferences, click .
+. Still in Preferences, click `Java->Editor->Save Actions`.
   Be sure the following options are selected:
 +
 * Perform the selected actions on save
@@ -133,30 +134,30 @@ If you cloned the project via git, download and install the Git plugin (EGit). A
 ==== HBase Project Setup in Eclipse using `m2eclipse`
 
 The easiest way is to use the +m2eclipse+ plugin for Eclipse.
-Eclipse Indigo or newer includes +m2eclipse+, or you can download it from link:http://www.eclipse.org/m2e//. It provides Maven integration for Eclipse, and even lets you use the direct Maven commands from within Eclipse to compile and test your project.
+Eclipse Indigo or newer includes +m2eclipse+, or you can download it from http://www.eclipse.org/m2e/. It provides Maven integration for Eclipse, and even lets you use the direct Maven commands from within Eclipse to compile and test your project.
 
 To import the project, click  and select the HBase root directory. `m2eclipse`                    locates all the hbase modules for you.
 
-If you install +m2eclipse+ and import HBase in your workspace, do the following to fix your eclipse Build Path. 
+If you install +m2eclipse+ and import HBase in your workspace, do the following to fix your eclipse Build Path.
 
 . Remove _target_ folder
 . Add _target/generated-jamon_ and _target/generated-sources/java_ folders.
 . Remove from your Build Path the exclusions on the _src/main/resources_ and _src/test/resources_ to avoid error message in the console, such as the following:
 +
 ----
-Failed to execute goal 
+Failed to execute goal
 org.apache.maven.plugins:maven-antrun-plugin:1.6:run (default) on project hbase:
-'An Ant BuildException has occured: Replace: source file .../target/classes/hbase-default.xml 
+'An Ant BuildException has occurred: Replace: source file .../target/classes/hbase-default.xml
 doesn't exist
 ----
 +
-This will also reduce the eclipse build cycles and make your life easier when developing. 
+This will also reduce the eclipse build cycles and make your life easier when developing.
 
 
 [[eclipse.commandline]]
 ==== HBase Project Setup in Eclipse Using the Command Line
 
-Instead of using `m2eclipse`, you can generate the Eclipse files from the command line. 
+Instead of using `m2eclipse`, you can generate the Eclipse files from the command line.
 
 . First, run the following command, which builds HBase.
   You only need to do this once.
@@ -181,7 +182,7 @@ mvn eclipse:eclipse
 The `$M2_REPO` classpath variable needs to be set up for the project.
 This needs to be set to your local Maven repository, which is usually _~/.m2/repository_
 
-If this classpath variable is not configured, you will see compile errors in Eclipse like this: 
+If this classpath variable is not configured, you will see compile errors in Eclipse like this:
 
 ----
 
@@ -209,14 +210,14 @@ Access restriction: The method getLong(Object, long) from the type Unsafe is not
 [[eclipse.more]]
 ==== Eclipse - More Information
 
-For additional information on setting up Eclipse for HBase development on Windows, see link:http://michaelmorello.blogspot.com/2011/09/hbase-subversion-eclipse-windows.html[Michael Morello's blog] on the topic. 
+For additional information on setting up Eclipse for HBase development on Windows, see link:http://michaelmorello.blogspot.com/2011/09/hbase-subversion-eclipse-windows.html[Michael Morello's blog] on the topic.
 
 === IntelliJ IDEA
 
-You can set up IntelliJ IDEA for similar functinoality as Eclipse.
+You can set up IntelliJ IDEA for similar functionality as Eclipse.
 Follow these steps.
 
-. Select 
+. Select
 . You do not need to select a profile.
   Be sure [label]#Maven project
   required# is selected, and click btn:[Next].
@@ -227,7 +228,7 @@ Using the Eclipse Code Formatter plugin for IntelliJ IDEA, you can import the HB
 
 === Other IDEs
 
-It would be userful to mirror the <<eclipse,eclipse>> set-up instructions for other IDEs.
+It would be useful to mirror the <<eclipse,eclipse>> set-up instructions for other IDEs.
 If you would like to assist, please have a look at link:https://issues.apache.org/jira/browse/HBASE-11704[HBASE-11704].
 
 [[build]]
@@ -237,20 +238,20 @@ If you would like to assist, please have a look at link:https://issues.apache.or
 === Basic Compile
 
 HBase is compiled using Maven.
-You must use Maven 3.x.
+You must use at least Maven 3.0.4.
 To check your Maven version, run the command +mvn -version+.
 
 .JDK Version Requirements
 [NOTE]
 ====
 Starting with HBase 1.0 you must use Java 7 or later to build from source code.
-See <<java,java>> for more complete information about supported JDK versions. 
+See <<java,java>> for more complete information about supported JDK versions.
 ====
 
 [[maven.build.commands]]
 ==== Maven Build Commands
 
-All commands are executed from the local HBase project directory. 
+All commands are executed from the local HBase project directory.
 
 ===== Package
 
@@ -269,7 +270,7 @@ mvn clean package -DskipTests
 ----
 
 With Eclipse set up as explained above in <<eclipse,eclipse>>, you can also use the menu:Build[] command in Eclipse.
-To create the full installable HBase package takes a little bit more work, so read on. 
+To create the full installable HBase package takes a little bit more work, so read on.
 
 [[maven.build.commands.compile]]
 ===== Compile
@@ -331,13 +332,13 @@ Tests may not all pass so you may need to pass `-DskipTests` unless you are incl
 ====
 You will see ERRORs like the above title if you pass the _default_ profile; e.g.
 if you pass +hadoop.profile=1.1+ when building 0.96 or +hadoop.profile=2.0+ when building hadoop 0.98; just drop the hadoop.profile stipulation in this case to get your build to run again.
-This seems to be a maven pecularity that is probably fixable but we've not spent the time trying to figure it.
+This seems to be a maven peculiarity that is probably fixable but we've not spent the time trying to figure it.
 ====
 
 Similarly, for 3.0, you would just replace the profile value.
-Note that Hadoop-3.0.0-SNAPSHOT does not currently have a deployed maven artificat - you will need to build and install your own in your local maven repository if you want to run against this profile. 
+Note that Hadoop-3.0.0-SNAPSHOT does not currently have a deployed maven artifact - you will need to build and install your own in your local maven repository if you want to run against this profile.
 
-In earilier versions of Apache HBase, you can build against older versions of Apache Hadoop, notably, Hadoop 0.22.x and 0.23.x.
+In earlier versions of Apache HBase, you can build against older versions of Apache Hadoop, notably, Hadoop 0.22.x and 0.23.x.
 If you are running, for example HBase-0.94 and wanted to build against Hadoop 0.23.x, you would run with:
 
 [source,bourne]
@@ -350,6 +351,10 @@ mvn -Dhadoop.profile=22 ...
 
 You may need to change the protobuf definitions that reside in the _hbase-protocol_ module or other modules.
 
+Previous to hbase-2.0.0, protobuf definition files were sprinkled across all hbase modules but now all
+to do with protobuf must reside in the hbase-protocol module; we are trying to contain our protobuf
+use so we can freely change versions without upsetting any downstream project use of protobuf.
+
 The protobuf files are located in _hbase-protocol/src/main/protobuf_.
 For the change to be effective, you will need to regenerate the classes.
 You can use maven profile `compile-protobuf` to do this.
@@ -367,7 +372,7 @@ You may also want to define `protoc.path` for the protoc binary, using the follo
 mvn compile -Pcompile-protobuf -Dprotoc.path=/opt/local/bin/protoc
 ----
 
-Read the _hbase-protocol/README.txt_ for more details. 
+Read the _hbase-protocol/README.txt_ for more details.
 
 [[build.thrift]]
 ==== Build Thrift
@@ -415,9 +420,9 @@ mvn -DskipTests package assembly:single deploy
 ==== Build Gotchas
 
 If you see `Unable to find resource 'VM_global_library.vm'`, ignore it.
-Its not an error.
+It's not an error.
 It is link:http://jira.codehaus.org/browse/MSITE-286[officially
-                        ugly] though. 
+                        ugly] though.
 
 [[releasing]]
 == Releasing Apache HBase
@@ -434,7 +439,7 @@ See <<java,java>> for Java requirements per HBase release.
 HBase 0.96.x will run on Hadoop 1.x or Hadoop 2.x.
 HBase 0.98 still runs on both, but HBase 0.98 deprecates use of Hadoop 1.
 HBase 1.x will _not_                run on Hadoop 1.
-In the following procedures, we make a distinction between HBase 1.x builds and the awkward process involved building HBase 0.96/0.98 for either Hadoop 1 or Hadoop 2 targets. 
+In the following procedures, we make a distinction between HBase 1.x builds and the awkward process involved building HBase 0.96/0.98 for either Hadoop 1 or Hadoop 2 targets.
 
 You must choose which Hadoop to build against.
 It is not possible to build a single HBase binary that runs against both Hadoop 1 and Hadoop 2.
@@ -450,6 +455,7 @@ You then reference these generated poms when you build.
 For now, just be aware of the difference between HBase 1.x builds and those of HBase 0.96-0.98.
 This difference is important to the build instructions.
 
+[[maven.settings.xml]]
 .Example _~/.m2/settings.xml_ File
 ====
 Publishing to maven requires you sign the artifacts you want to upload.
@@ -499,36 +505,39 @@ For the build to sign them for you, you a properly configured _settings.xml_ in
 === Making a Release Candidate
 
 NOTE: These instructions are for building HBase 1.0.x.
-For building earlier versions, the process is different.
-See this section under the respective release documentation folders. 
+For building earlier versions, e.g. 0.98.x, the process is different.
+See this section under the respective release documentation folders.
 
 .Point Releases
-If you are making a point release (for example to quickly address a critical incompatability or security problem) off of a release branch instead of a development branch, the tagging instructions are slightly different.
-I'll prefix those special steps with _Point Release Only_. 
+If you are making a point release (for example to quickly address a critical incompatibility or security problem) off of a release branch instead of a development branch, the tagging instructions are slightly different.
+I'll prefix those special steps with _Point Release Only_.
 
 .Before You Begin
 Before you make a release candidate, do a practice run by deploying a snapshot.
 Before you start, check to be sure recent builds have been passing for the branch from where you are going to take your release.
-You should also have tried recent branch tips out on a cluster under load, perhaps by running the `hbase-it` integration test suite for a few hours to 'burn in' the near-candidate bits. 
+You should also have tried recent branch tips out on a cluster under load, perhaps by running the `hbase-it` integration test suite for a few hours to 'burn in' the near-candidate bits.
 
 .Point Release Only
 [NOTE]
 ====
 At this point you should tag the previous release branch (ex: 0.96.1) with the new point release tag (e.g.
-0.96.1.1 tag). Any commits with changes for the point release should be appled to the new tag. 
+0.96.1.1 tag). Any commits with changes for the point release should go against the new tag.
 ====
 
 The Hadoop link:http://wiki.apache.org/hadoop/HowToRelease[How To
-                    Release] wiki page is used as a model for most of the instructions below, and may have more detail on particular sections, so it is worth review.
+                    Release] wiki page is used as a model for most of the instructions below.
+                    Although it now stale, it may have more detail on particular sections, so
+                    it is worth review especially if you get stuck.
 
 .Specifying the Heap Space for Maven on OSX
 [NOTE]
 ====
-On OSX, you may need to specify the heap space for Maven commands, by setting the `MAVEN_OPTS` variable to `-Xmx3g`.
+On OSX, you may run into OutOfMemoryErrors building, particularly building the site and
+documentation. Up the heap and permgen space for Maven by setting the `MAVEN_OPTS` variable.
 You can prefix the variable to the Maven command, as in the following example:
 
 ----
-MAVEN_OPTS="-Xmx2g" mvn package
+MAVEN_OPTS="-Xmx4g -XX:MaxPermSize=256m" mvn package
 ----
 
 You could also set this in an environment variable or alias in your shell.
@@ -546,7 +555,8 @@ The script handles everything else, and comes in handy.
 Update _CHANGES.txt_ with the changes since the last release.
 Make sure the URL to the JIRA points to the proper location which lists fixes for this release.
 Adjust the version in all the POM files appropriately.
-If you are making a release candidate, you must remove the `-SNAPSHOT` label from all versions.
+If you are making a release candidate, you must remove the `-SNAPSHOT` label from all versions
+in all pom.xml files.
 If you are running this receipe to publish a snapshot, you must keep the `-SNAPSHOT` suffix on the hbase version.
 The link:http://mojo.codehaus.org/versions-maven-plugin/[Versions
                             Maven Plugin] can be of use here.
@@ -558,12 +568,13 @@ To set a version in all the many poms of the hbase multi-module project, use a c
 $ mvn clean org.codehaus.mojo:versions-maven-plugin:1.3.1:set -DnewVersion=0.96.0
 ----
 +
-Checkin the _CHANGES.txt_ and any version changes.
+Make sure all versions in poms are changed! Checkin the _CHANGES.txt_ and any version changes.
 
 . Update the documentation.
 +
-Update the documentation under _src/main/docbkx_.
-This usually involves copying the latest from trunk and making version-particular adjustments to suit this release candidate version. 
+Update the documentation under _src/main/asciidoc_.
+This usually involves copying the latest from master and making version-particular
+adjustments to suit this release candidate version.
 
 . Build the source tarball.
 +
@@ -582,8 +593,8 @@ $ mvn clean install -DskipTests assembly:single -Dassembly.file=hbase-assembly/s
 Extract the tarball and make sure it looks good.
 A good test for the src tarball being 'complete' is to see if you can build new tarballs from this source bundle.
 If the source tarball is good, save it off to a _version directory_, a directory somewhere where you are collecting all of the tarballs you will publish as part of the release candidate.
-For example if you were building a hbase-0.96.0 release candidate, you might call the directory _hbase-0.96.0RC0_.
-Later you will publish this directory as our release candidate up on http://people.apache.org/~YOU. 
+For example if you were building an hbase-0.96.0 release candidate, you might call the directory _hbase-0.96.0RC0_.
+Later you will publish this directory as our release candidate.
 
 . Build the binary tarball.
 +
@@ -609,12 +620,13 @@ $ mvn install -DskipTests site assembly:single -Prelease
 ----
 
 +
-Otherwise, the build complains that hbase modules are not in the maven repository when you try to do it at once, especially on fresh repository.
+Otherwise, the build complains that hbase modules are not in the maven repository
+when you try to do it all in one step, especially on a fresh repository.
 It seems that you need the install goal in both steps.
 +
 Extract the generated tarball and check it out.
 Look at the documentation, see if it runs, etc.
-If good, copy the tarball to the above mentioned _version directory_. 
+If good, copy the tarball to the above mentioned _version directory_.
 
 . Create a new tag.
 +
@@ -631,33 +643,35 @@ Release needs to be tagged for the next step.
 . Deploy to the Maven Repository.
 +
 Next, deploy HBase to the Apache Maven repository, using the `apache-release` profile instead of the `release` profile when running the `mvn deploy` command.
-This profile invokes the Apache pom referenced by our pom files, and also signs your artifacts published to Maven, as long as the _settings.xml_ is configured correctly, as described in <<mvn.settings.file,mvn.settings.file>>.
+This profile invokes the Apache pom referenced by our pom files, and also signs your artifacts published to Maven, as long as the _settings.xml_ is configured correctly, as described in <<maven.settings.xml>>.
 +
 [source,bourne]
 ----
 
-$ mvn deploy -DskipTests -Papache-release
+$ mvn deploy -DskipTests -Papache-release -Prelease
 ----
 +
 This command copies all artifacts up to a temporary staging Apache mvn repository in an 'open' state.
-More work needs to be done on these maven artifacts to make them generally available. 
+More work needs to be done on these maven artifacts to make them generally available.
 +
-We do not release HBase tarball to the Apache Maven repository. To avoid deploying the tarball, do not include the `assembly:single` goal in your `mvn deploy` command. Check the deployed artifacts as described in the next section.
+We do not release HBase tarball to the Apache Maven repository. To avoid deploying the tarball, do not
+include the `assembly:single` goal in your `mvn deploy` command. Check the deployed artifacts as described in the next section.
 
 . Make the Release Candidate available.
 +
 The artifacts are in the maven repository in the staging area in the 'open' state.
 While in this 'open' state you can check out what you've published to make sure all is good.
-To do this, login at link:http://repository.apache.org[repository.apache.org]                        using your Apache ID.
-Find your artifacts in the staging repository.
-Browse the content.
-Make sure all artifacts made it up and that the poms look generally good.
-If it checks out, 'close' the repo.
-This will make the artifacts publically available.
-You will receive an email with the URL to give out for the temporary staging repository for others to use trying out this new release candidate.
-Include it in the email that announces the release candidate.
-Folks will need to add this repo URL to their local poms or to their local _settings.xml_ file to pull the published release candidate artifacts.
-If the published artifacts are incomplete or have problems, just delete the 'open' staged artifacts.
+To do this, log in to Apache's Nexus at link:http://repository.apache.org[repository.apache.org] using your Apache ID.
+Find your artifacts in the staging repository. Click on 'Staging Repositories' and look for a new one ending in "hbase" with a status of 'Open', select it.
+Use the tree view to expand the list of repository contents and inspect if the artifacts you expect are present. Check the POMs.
+As long as the staging repo is open you can re-upload if something is missing or built incorrectly.
++
+If something is seriously wrong and you would like to back out the upload, you can use the 'Drop' button to drop and delete the staging repository.
++
+If it checks out, close the repo using the 'Close' button. The repository must be closed before a public URL to it becomes available. It may take a few minutes for the repository to close. Once complete you'll see a public URL to the repository in the Nexus UI. You may also receive an email with the URL. Provide the URL to the temporary staging repository in the email that announces the release candidate.
+(Folks will need to add this repo URL to their local poms or to their local _settings.xml_ file to pull the published release candidate artifacts.)
++
+When the release vote concludes successfully, return here and click the 'Release' button to release the artifacts to central. The release process will automatically drop and delete the staging repository.
 +
 .hbase-downstreamer
 [NOTE]
@@ -665,7 +679,7 @@ If the published artifacts are incomplete or have problems, just delete the 'ope
 See the link:https://github.com/saintstack/hbase-downstreamer[hbase-downstreamer] test for a simple example of a project that is downstream of HBase an depends on it.
 Check it out and run its simple test to make sure maven artifacts are properly deployed to the maven repository.
 Be sure to edit the pom to point to the proper staging repository.
-Make sure you are pulling from the repository when tests run and that you are not getting from your local repository, by either passing the `-U` flag or deleting your local repo content and check maven is pulling from remote out of the staging repository. 
+Make sure you are pulling from the repository when tests run and that you are not getting from your local repository, by either passing the `-U` flag or deleting your local repo content and check maven is pulling from remote out of the staging repository.
 ====
 +
 See link:http://www.apache.org/dev/publishing-maven-artifacts.html[Publishing Maven Artifacts] for some pointers on this maven staging process.
@@ -673,7 +687,7 @@ See link:http://www.apache.org/dev/publishing-maven-artifacts.html[Publishing Ma
 NOTE: We no longer publish using the maven release plugin.
 Instead we do +mvn deploy+.
 It seems to give us a backdoor to maven release publishing.
-If there is no _-SNAPSHOT_                            on the version string, then we are 'deployed' to the apache maven repository staging directory from which we can publish URLs for candidates and later, if they pass, publish as release (if a _-SNAPSHOT_ on the version string, deploy will put the artifacts up into apache snapshot repos). 
+If there is no _-SNAPSHOT_ on the version string, then we are 'deployed' to the apache maven repository staging directory from which we can publish URLs for candidates and later, if they pass, publish as release (if a _-SNAPSHOT_ on the version string, deploy will put the artifacts up into apache snapshot repos).
 +
 If the HBase version ends in `-SNAPSHOT`, the artifacts go elsewhere.
 They are put into the Apache snapshots repository directly and are immediately available.
@@ -687,15 +701,9 @@ These are publicly accessible in a temporary staging repository whose URL you sh
 The above mentioned script, _make_rc.sh_ does all of the above for you minus the check of the artifacts built, the closing of the staging repository up in maven, and the tagging of the release.
 If you run the script, do your checks at this stage verifying the src and bin tarballs and checking what is up in staging using hbase-downstreamer project.
 Tag before you start the build.
-You can always delete it if the build goes haywire. 
+You can always delete it if the build goes haywire.
 
-. Sign, upload, and 'stage' your version directory to link:http://people.apache.org[people.apache.org] (TODO:
-  There is a new location to stage releases using svnpubsub.  See
-  (link:https://issues.apache.org/jira/browse/HBASE-10554[HBASE-10554 Please delete old releases from mirroring system]).
-+
-If all checks out, next put the _version directory_ up on link:http://people.apache.org[people.apache.org].
-You will need to sign and fingerprint them before you push them up.
-In the _version directory_ run the following commands: 
+. Sign, fingerprint and then 'stage' your release candiate version directory via svnpubsub by committing your directory to link:https://dist.apache.org/repos/dist/dev/hbase/[The 'dev' distribution directory] (See comments on link:https://issues.apache.org/jira/browse/HBASE-10554[HBASE-10554 Please delete old releases from mirroring system] but in essence it is an svn checkout of https://dist.apache.org/repos/dist/dev/hbase -- releases are at https://dist.apache.org/repos/dist/release/hbase). In the _version directory_ run the following commands:
 +
 [source,bourne]
 ----
@@ -705,18 +713,35 @@ $ for i in *.tar.gz; do echo $i; gpg --print-md MD5 $i > $i.md5 ; done
 $ for i in *.tar.gz; do echo $i; gpg --print-md SHA512 $i > $i.sha ; done
 $ for i in *.tar.gz; do echo $i; gpg --armor --output $i.asc --detach-sig $i  ; done
 $ cd ..
-# Presuming our 'version directory' is named 0.96.0RC0, now copy it up to people.apache.org.
-$ rsync -av 0.96.0RC0 people.apache.org:public_html
+# Presuming our 'version directory' is named 0.96.0RC0, copy it to the svn checkout of the dist dev dir
+# in this case named hbase.dist.dev.svn
+$ cd /Users/stack/checkouts/hbase.dist.dev.svn
+$ svn info
+Path: .
+Working Copy Root Path: /Users/stack/checkouts/hbase.dist.dev.svn
+URL: https://dist.apache.org/repos/dist/dev/hbase
+Repository Root: https://dist.apache.org/repos/dist
+Repository UUID: 0d268c88-bc11-4956-87df-91683dc98e59
+Revision: 15087
+Node Kind: directory
+Schedule: normal
+Last Changed Author: ndimiduk
+Last Changed Rev: 15045
+Last Changed Date: 2016-08-28 11:13:36 -0700 (Sun, 28 Aug 2016)
+$ mv 0.96.0RC0 /Users/stack/checkouts/hbase.dist.dev.svn
+$ svn add 0.96.0RC0
+$ svn commit ...
 ----
 +
-Make sure the link:http://people.apache.org[people.apache.org] directory is showing and that the mvn repo URLs are good.
-Announce the release candidate on the mailing list and call a vote. 
+Ensure it actually gets published by checking link:https://dist.apache.org/repos/dist/dev/hbase/[https://dist.apache.org/repos/dist/dev/hbase/].
+
+Announce the release candidate on the mailing list and call a vote.
 
 
 [[maven.snapshot]]
 === Publishing a SNAPSHOT to maven
 
-Make sure your _settings.xml_ is set up properly, as in <<mvn.settings.file,mvn.settings.file>>.
+Make sure your _settings.xml_ is set up properly (see <<maven.settings.xml>>).
 Make sure the hbase version includes `-SNAPSHOT` as a suffix.
 Following is an example of publishing SNAPSHOTS of a release that had an hbase version of 0.96.0 in its poms.
 
@@ -729,7 +754,7 @@ Following is an example of publishing SNAPSHOTS of a release that had an hbase v
 
 The _make_rc.sh_ script mentioned above (see <<maven.release,maven.release>>) can help you publish `SNAPSHOTS`.
 Make sure your `hbase.version` has a `-SNAPSHOT`                suffix before running the script.
-It will put a snapshot up into the apache snapshot repository for you. 
+It will put a snapshot up into the apache snapshot repository for you.
 
 [[hbase.rc.voting]]
 == Voting on Release Candidates
@@ -744,7 +769,7 @@ PMC members, please read this WIP doc on policy voting for a release candidate,
                 requirements of the ASF policy on releases._ Regards the latter, run +mvn apache-rat:check+ to verify all files are suitably licensed.
 See link:http://search-hadoop.com/m/DHED4dhFaU[HBase, mail # dev - On
                 recent discussion clarifying ASF release policy].
-for how we arrived at this process. 
+for how we arrived at this process.
 
 [[documentation]]
 == Generating the HBase Reference Guide
@@ -752,7 +777,7 @@ for how we arrived at this process.
 The manual is marked up using Asciidoc.
 We then use the link:http://asciidoctor.org/docs/asciidoctor-maven-plugin/[Asciidoctor maven plugin] to transform the markup to html.
 This plugin is run when you specify the +site+ goal as in when you run +mvn site+.
-See <<appendix_contributing_to_documentation,appendix contributing to documentation>> for more information on building the documentation. 
+See <<appendix_contributing_to_documentation,appendix contributing to documentation>> for more information on building the documentation.
 
 [[hbase.org]]
 == Updating link:http://hbase.apache.org[hbase.apache.org]
@@ -765,24 +790,7 @@ See <<appendix_contributing_to_documentation,appendix contributing to documentat
 [[hbase.org.site.publishing]]
 === Publishing link:http://hbase.apache.org[hbase.apache.org]
 
-As of link:https://issues.apache.org/jira/browse/INFRA-5680[INFRA-5680 Migrate apache hbase website], to publish the website, build it using Maven, and then deploy it over a checkout of _https://svn.apache.org/repos/asf/hbase/hbase.apache.org/trunk_                and check in your changes.
-The script _dev-scripts/publish_hbase_website.sh_ is provided to automate this process and to be sure that stale files are removed from SVN.
-Review the script even if you decide to publish the website manually.
-Use the script as follows:
-
-----
-$ publish_hbase_website.sh -h
-Usage: publish_hbase_website.sh [-i | -a] [-g <dir>] [-s <dir>]
- -h          Show this message
- -i          Prompts the user for input
- -a          Does not prompt the user. Potentially dangerous.
- -g          The local location of the HBase git repository
- -s          The local location of the HBase svn checkout
- Either --interactive or --silent is required.
- Edit the script to set default Git and SVN directories.
-----
-
-NOTE: The SVN commit takes a long time.
+See <<website_publish>> for instructions on publishing the website and documentation.
 
 [[hbase.tests]]
 == Tests
@@ -806,7 +814,7 @@ For any other module, for example `hbase-common`, the tests must be strict unit
 
 The HBase shell and its tests are predominantly written in jruby.
 In order to make these tests run as a part of the standard build, there is a single JUnit test, `TestShell`, that takes care of loading the jruby implemented tests and running them.
-You can run all of these tests from the top level with: 
+You can run all of these tests from the top level with:
 
 [source,bourne]
 ----
@@ -816,7 +824,7 @@ You can run all of these tests from the top level with:
 
 Alternatively, you may limit the shell tests that run using the system variable `shell.test`.
 This value should specify the ruby literal equivalent of a particular test case by name.
-For example, the tests that cover the shell commands for altering tables are contained in the test case `AdminAlterTableTest`        and you can run them with: 
+For example, the tests that cover the shell commands for altering tables are contained in the test case `AdminAlterTableTest`        and you can run them with:
 
 [source,bourne]
 ----
@@ -826,7 +834,7 @@ For example, the tests that cover the shell commands for altering tables are con
 
 You may also use a link:http://docs.ruby-doc.com/docs/ProgrammingRuby/html/language.html#UJ[Ruby Regular Expression
       literal] (in the `/pattern/` style) to select a set of test cases.
-You can run all of the HBase admin related tests, including both the normal administration and the security administration, with the command: 
+You can run all of the HBase admin related tests, including both the normal administration and the security administration, with the command:
 
 [source,bourne]
 ----
@@ -834,7 +842,7 @@ You can run all of the HBase admin related tests, including both the normal admi
       mvn clean test -Dtest=TestShell -Dshell.test=/.*Admin.*Test/
 ----
 
-In the event of a test failure, you can see details by examining the XML version of the surefire report results 
+In the event of a test failure, you can see details by examining the XML version of the surefire report results
 
 [source,bourne]
 ----
@@ -876,7 +884,8 @@ Also, keep in mind that if you are running tests in the `hbase-server` module yo
 [[hbase.unittests]]
 === Unit Tests
 
-Apache HBase unit tests are subdivided into four categories: small, medium, large, and integration with corresponding JUnit link:http://www.junit.org/node/581[categories]: `SmallTests`, `MediumTests`, `LargeTests`, `IntegrationTests`.
+Apache HBase test cases are subdivided into four categories: small, medium, large, and
+integration with corresponding JUnit link:http://www.junit.org/node/581[categories]: `SmallTests`, `MediumTests`, `LargeTests`, `IntegrationTests`.
 JUnit categories are denoted using java annotations and look like this in your unit test code.
 
 [source,java]
@@ -891,51 +900,53 @@ public class TestHRegionInfo {
 }
 ----
 
-The above example shows how to mark a unit test as belonging to the `small` category.
-All unit tests in HBase have a categorization. 
+The above example shows how to mark a test case as belonging to the `small` category.
+All test cases in HBase should have a categorization.
 
-The first three categories, `small`, `medium`, and `large`, are for tests run when you type `$ mvn test`.
+The first three categories, `small`, `medium`, and `large`, are for test cases which run when you
+type `$ mvn test`.
 In other words, these three categorizations are for HBase unit tests.
 The `integration` category is not for unit tests, but for integration tests.
 These are run when you invoke `$ mvn verify`.
 Integration tests are described in <<integration.tests,integration.tests>>.
 
-HBase uses a patched maven surefire plugin and maven profiles to implement its unit test characterizations. 
+HBase uses a patched maven surefire plugin and maven profiles to implement its unit test characterizations.
 
-Keep reading to figure which annotation of the set small, medium, and large to put on your new HBase unit test. 
+Keep reading to figure which annotation of the set small, medium, and large to put on your new
+HBase test case.
 
 .Categorizing Tests
 Small Tests (((SmallTests)))::
-  _Small_ tests are executed in a shared JVM.
-  We put in this category all the tests that can be executed quickly in a shared JVM.
-  The maximum execution time for a small test is 15 seconds, and small tests should not use a (mini)cluster.
+  _Small_ test cases are executed in a shared JVM and individual test cases should run in 15 seconds
+   or less; i.e. a link:https://en.wikipedia.org/wiki/JUnit[junit test fixture], a java object made
+   up of test methods, should finish in under 15 seconds. These test cases can not use mini cluster.
+   These are run as part of patch pre-commit.
 
 Medium Tests (((MediumTests)))::
-  _Medium_ tests represent tests that must be executed before proposing a patch.
-  They are designed to run in less than 30 minutes altogether, and are quite stable in their results.
-  They are designed to last less than 50 seconds individually.
-  They can use a cluster, and each of them is executed in a separate JVM. 
+  _Medium_ test cases are executed in separate JVM and individual test case should run in 50 seconds
+   or less. Together, they should take less than 30 minutes, and are quite stable in their results.
+   These test cases can use a mini cluster. These are run as part of patch pre-commit.
 
 Large Tests (((LargeTests)))::
-  _Large_ tests are everything else.
+  _Large_ test cases are everything else.
   They are typically large-scale tests, regression tests for specific bugs, timeout tests, performance tests.
   They are executed before a commit on the pre-integration machines.
-  They can be run on the developer machine as well. 
+  They can be run on the developer machine as well.
 
 Integration Tests (((IntegrationTests)))::
   _Integration_ tests are system level tests.
-  See <<integration.tests,integration.tests>> for more info. 
+  See <<integration.tests,integration.tests>> for more info.
 
 [[hbase.unittests.cmds]]
 === Running tests
 
 [[hbase.unittests.cmds.test]]
-==== Default: small and medium category tests 
+==== Default: small and medium category tests
 
 Running `mvn test` will execute all small tests in a single JVM (no fork) and then medium tests in a separate JVM for each test instance.
 Medium tests are NOT executed if there is an error in a small test.
 Large tests are NOT executed.
-There is one report for small tests, and one report for medium tests if they are executed. 
+There is one report for small tests, and one report for medium tests if they are executed.
 
 [[hbase.unittests.cmds.test.runalltests]]
 ==== Running all tests
@@ -943,38 +954,38 @@ There is one report for small tests, and one report for medium tests if they are
 Running `mvn test -P runAllTests` will execute small tests in a single JVM then medium and large tests in a separate JVM for each test.
 Medium and large tests are NOT executed if there is an error in a small test.
 Large tests are NOT executed if there is an error in a small or medium test.
-There is one report for small tests, and one report for medium and large tests if they are executed. 
+There is one report for small tests, and one report for medium and large tests if they are executed.
 
 [[hbase.unittests.cmds.test.localtests.mytest]]
 ==== Running a single test or all tests in a package
 
-To run an individual test, e.g. `MyTest`, rum `mvn test -Dtest=MyTest` You can also pass multiple, individual tests as a comma-delimited list: 
+To run an individual test, e.g. `MyTest`, rum `mvn test -Dtest=MyTest` You can also pass multiple, individual tests as a comma-delimited list:
 [source,bash]
 ----
 mvn test  -Dtest=MyTest1,MyTest2,MyTest3
 ----
-You can also pass a package, which will run all tests under the package: 
+You can also pass a package, which will run all tests under the package:
 [source,bash]
 ----
 mvn test '-Dtest=org.apache.hadoop.hbase.client.*'
-----                
+----
 
 When `-Dtest` is specified, the `localTests` profile will be used.
 It will use the official release of maven surefire, rather than our custom surefire plugin, and the old connector (The HBase build uses a patched version of the maven surefire plugin). Each junit test is executed in a separate JVM (A fork per test class). There is no parallelization when tests are running in this mode.
 You will see a new message at the end of the -report: `"[INFO] Tests are skipped"`.
 It's harmless.
-However, you need to make sure the sum of `Tests run:` in the `Results:` section of test reports matching the number of tests you specified because no error will be reported when a non-existent test case is specified. 
+However, you need to make sure the sum of `Tests run:` in the `Results:` section of test reports matching the number of tests you specified because no error will be reported when a non-existent test case is specified.
 
 [[hbase.unittests.cmds.test.profiles]]
 ==== Other test invocation permutations
 
-Running `mvn test -P runSmallTests` will execute "small" tests only, using a single JVM. 
+Running `mvn test -P runSmallTests` will execute "small" tests only, using a single JVM.
 
-Running `mvn test -P runMediumTests` will execute "medium" tests only, launching a new JVM for each test-class. 
+Running `mvn test -P runMediumTests` will execute "medium" tests only, launching a new JVM for each test-class.
 
-Running `mvn test -P runLargeTests` will execute "large" tests only, launching a new JVM for each test-class. 
+Running `mvn test -P runLargeTests` will execute "large" tests only, launching a new JVM for each test-class.
 
-For convenience, you can run `mvn test -P runDevTests` to execute both small and medium tests, using a single JVM. 
+For convenience, you can run `mvn test -P runDevTests` to execute both small and medium tests, using a single JVM.
 
 [[hbase.unittests.test.faster]]
 ==== Running tests faster
@@ -996,7 +1007,7 @@ $ sudo mkdir /ram2G
 sudo mount -t tmpfs -o size=2048M tmpfs /ram2G
 ----
 
-You can then use it to run all HBase tests on 2.0 with the command: 
+You can then use it to run all HBase tests on 2.0 with the command:
 
 ----
 mvn test
@@ -1004,7 +1015,7 @@ mvn test
                         -Dtest.build.data.basedirectory=/ram2G
 ----
 
-On earlier versions, use: 
+On earlier versions, use:
 
 ----
 mvn test
@@ -1023,7 +1034,7 @@ It must be executed from the directory which contains the _pom.xml_.
 For example running +./dev-support/hbasetests.sh+ will execute small and medium tests.
 Running +./dev-support/hbasetests.sh
                         runAllTests+ will execute all tests.
-Running +./dev-support/hbasetests.sh replayFailed+ will rerun the failed tests a second time, in a separate jvm and without parallelisation. 
+Running +./dev-support/hbasetests.sh replayFailed+ will rerun the failed tests a second time, in a separate jvm and without parallelisation.
 
 [[hbase.unittests.resource.checker]]
 ==== Test Resource Checker(((Test ResourceChecker)))
@@ -1033,7 +1044,7 @@ Check the _*-out.txt_ files). The resources counted are the number of threads, t
 If the number has increased, it adds a _LEAK?_ comment in the logs.
 As you can have an HBase instance running in the background, some threads can be deleted/created without any specific action in the test.
 However, if the test does not work as expected, or if the test should not impact these resources, it's worth checking these log lines [computeroutput]+...hbase.ResourceChecker(157): before...+                    and [computeroutput]+...hbase.ResourceChecker(157): after...+.
-For example: 
+For example:
 
 ----
 2012-09-26 09:22:15,315 INFO [pool-1-thread-1]
@@ -1061,9 +1072,7 @@ ConnectionCount=1 (was 1)
 
 * All tests must be categorized, if not they could be skipped.
 * All tests should be written to be as fast as possible.
-* Small category tests should last less than 15 seconds, and must not have any side effect.
-* Medium category tests should last less than 50 seconds.
-* Large category tests should last less than 3 minutes.
+* See <<hbase.unittests,hbase.unittests> for test case categories and corresponding timeouts.
   This should ensure a good parallelization for people using it, and ease the analysis when the test fails.
 
 [[hbase.tests.sleeps]]
@@ -1076,10 +1085,10 @@ This allows understanding what the test is waiting for.
 Moreover, the test will work whatever the machine performance is.
 Sleep should be minimal to be as fast as possible.
 Waiting for a variable should be done in a 40ms sleep loop.
-Waiting for a socket operation should be done in a 200 ms sleep loop. 
+Waiting for a socket operation should be done in a 200 ms sleep loop.
 
 [[hbase.tests.cluster]]
-==== Tests using a cluster 
+==== Tests using a cluster
 
 Tests using a HRegion do not have to start a cluster: A region can use the local file system.
 Start/stopping a cluster cost around 10 seconds.
@@ -1087,7 +1096,48 @@ They should not be started per test method but per test class.
 Started cluster must be shutdown using [method]+HBaseTestingUtility#shutdownMiniCluster+, which cleans the directories.
 As most as possible, tests should use the default settings for the cluster.
 When they don't, they should document it.
-This will allow to share the cluster later. 
+This will allow to share the cluster later.
+
+[[hbase.tests.example.code]]
+==== Tests Skeleton Code
+
+Here is a test skeleton code with Categorization and a Category-based timeout rule to copy and paste and use as basis for test contribution.
+[source,java]
+----
+/**
+ * Describe what this testcase tests. Talk about resources initialized in @BeforeClass (before
+ * any test is run) and before each test is run, etc.
+ */
+// Specify the category as explained in <<hbase.unittests,hbase.unittests>>.
+@Category(SmallTests.class)
+public class TestExample {
+  // Replace the TestExample.class in the below with the name of your test fixture class.
+  private static final Log LOG = LogFactory.getLog(TestExample.class);
+
+  // Handy test rule that allows you subsequently get the name of the current method. See
+  // down in 'testExampleFoo()' where we use it to log current test's name.
+  @Rule public TestName testName = new TestName();
+
+  // CategoryBasedTimeout.forClass(<testcase>) decides the timeout based on the category
+  // (small/medium/large) of the testcase. @ClassRule requires that the full testcase runs within
+  // this timeout irrespective of individual test methods' times.
+  @ClassRule
+  public static TestRule timeout = CategoryBasedTimeout.forClass(TestExample.class);
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testExampleFoo() {
+    LOG.info("Running test " + testName.getMethodName());
+  }
+}
+----
 
 [[integration.tests]]
 === Integration Tests
@@ -1095,16 +1145,16 @@ This will allow to share the cluster later.
 HBase integration/system tests are tests that are beyond HBase unit tests.
 They are generally long-lasting, sizeable (the test can be asked to 1M rows or 1B rows), targetable (they can take configuration that will point them at the ready-made cluster they are to run against; integration tests do not include cluster start/stop code), and verifying success, integration tests rely on public APIs only; they do not attempt to examine server internals asserting success/fail.
 Integration tests are what you would run when you need to more elaborate proofing of a release candidate beyond what unit tests can do.
-They are not generally run on the Apache Continuous Integration build server, however, some sites opt to run integration tests as a part of their continuous testing on an actual cluster. 
+They are not generally run on the Apache Continuous Integration build server, however, some sites opt to run integration tests as a part of their continuous testing on an actual cluster.
 
 Integration tests currently live under the _src/test_                directory in the hbase-it submodule and will match the regex: _**/IntegrationTest*.java_.
-All integration tests are also annotated with `@Category(IntegrationTests.class)`. 
+All integration tests are also annotated with `@Category(IntegrationTests.class)`.
 
 Integration tests can be run in two modes: using a mini cluster, or against an actual distributed cluster.
 Maven failsafe is used to run the tests using the mini cluster.
 IntegrationTestsDriver class is used for executing the tests against a distributed cluster.
 Integration tests SHOULD NOT assume that they are running against a mini cluster, and SHOULD NOT use private API's to access cluster state.
-To interact with the distributed or mini cluster uniformly, `IntegrationTestingUtility`, and `HBaseCluster` classes, and public client API's can be used. 
+To interact with the distributed or mini cluster uniformly, `IntegrationTestingUtility`, and `HBaseCluster` classes, and public client API's can be used.
 
 On a distributed cluster, integration tests that use ChaosMonkey or otherwise manipulate services thru cluster manager (e.g.
 restart regionservers) use SSH to do it.
@@ -1118,15 +1168,15 @@ The argument 1 (%1$s) is SSH options set the via opts setting or via environment
 ----
 /usr/bin/ssh %1$s %2$s%3$s%4$s "su hbase - -c \"%5$s\""
 ----
-That way, to kill RS (for example) integration tests may run: 
+That way, to kill RS (for example) integration tests may run:
 [source,bash]
 ----
 {/usr/bin/ssh some-hostname "su hbase - -c \"ps aux | ... | kill ...\""}
 ----
-The command is logged in the test logs, so you can verify it is correct for your environment. 
+The command is logged in the test logs, so you can verify it is correct for your environment.
 
 To disable the running of Integration Tests, pass the following profile on the command line `-PskipIntegrationTests`.
-For example, 
+For example,
 [source]
 ----
 $ mvn clean install test -Dtest=TestZooKeeper  -PskipIntegrationTests
@@ -1148,9 +1198,9 @@ mvn verify
 ----
 
 If you just want to run the integration tests in top-level, you need to run two commands.
-First: +mvn failsafe:integration-test+ This actually runs ALL the integration tests. 
+First: +mvn failsafe:integration-test+ This actually runs ALL the integration tests.
 
-NOTE: This command will always output `BUILD SUCCESS` even if there are test failures. 
+NOTE: This command will always output `BUILD SUCCESS` even if there are test failures.
 
 At this point, you could grep the output by hand looking for failed tests.
 However, maven will do this for us; just use: +mvn
@@ -1161,19 +1211,19 @@ However, maven will do this for us; just use: +mvn
 
 This is very similar to how you specify running a subset of unit tests (see above), but use the property `it.test` instead of `test`.
 To just run `IntegrationTestClassXYZ.java`, use: +mvn
-                            failsafe:integration-test -Dit.test=IntegrationTestClassXYZ+                        The next thing you might want to do is run groups of integration tests, say all integration tests that are named IntegrationTestClassX*.java: +mvn failsafe:integration-test -Dit.test=*ClassX*+ This runs everything that is an integration test that matches *ClassX*. This means anything matching: "**/IntegrationTest*ClassX*". You can also run multiple groups of integration tests using comma-delimited lists (similar to unit tests). Using a list of matches still supports full regex matching for each of the groups.This would look something like: +mvn
-                            failsafe:integration-test -Dit.test=*ClassX*, *ClassY+                    
+                            failsafe:integration-test -Dit.test=IntegrationTestClassXYZ+                        The next thing you might want to do is run groups of integration tests, say all integration tests that are named IntegrationTestClassX*.java: +mvn failsafe:integration-test -Dit.test=*ClassX*+ This runs everything that is an integration test that matches *ClassX*. This means anything matching: "**/IntegrationTest*ClassX*". You can also run multiple groups of integration tests using comma-delimited lists (similar to unit tests). Using a list of matches still supports full regex matching for each of the groups. This would look something like: +mvn
+                            failsafe:integration-test -Dit.test=*ClassX*, *ClassY+
 
 [[maven.build.commands.integration.tests.distributed]]
 ==== Running integration tests against distributed cluster
 
 If you have an already-setup HBase cluster, you can launch the integration tests by invoking the class `IntegrationTestsDriver`.
 You may have to run test-compile first.
-The configuration will be picked by the bin/hbase script. 
+The configuration will be picked by the bin/hbase script.
 [source,bourne]
 ----
 mvn test-compile
----- 
+----
 Then launch the tests with:
 
 [source,bourne]
@@ -1186,26 +1236,30 @@ Running the IntegrationTestsDriver without any argument will launch tests found
 See the usage, by passing -h, to see how to filter test classes.
 You can pass a regex which is checked against the full class name; so, part of class name can be used.
 IntegrationTestsDriver uses Junit to run the tests.
-Currently there is no support for running integration tests against a distributed cluster using maven (see link:https://issues.apache.org/jira/browse/HBASE-6201[HBASE-6201]). 
+Currently there is no support for running integration tests against a distributed cluster using maven (see link:https://issues.apache.org/jira/browse/HBASE-6201[HBASE-6201]).
 
 The tests interact with the distributed cluster by using the methods in the `DistributedHBaseCluster` (implementing `HBaseCluster`) class, which in turn uses a pluggable `ClusterManager`.
 Concrete implementations provide actual functionality for carrying out deployment-specific and environment-dependent tasks (SSH, etc). The default `ClusterManager` is `HBaseClusterManager`, which uses SSH to remotely execute start/stop/kill/signal commands, and assumes some posix commands (ps, etc). Also assumes the user running the test has enough "power" to start/stop servers on the remote machines.
 By default, it picks up `HBASE_SSH_OPTS`, `HBASE_HOME`, `HBASE_CONF_DIR` from the env, and uses `bin/hbase-daemon.sh` to carry out the actions.
 Currently tarball deployments, deployments which uses _hbase-daemons.sh_, and link:http://incubator.apache.org/ambari/[Apache Ambari]                    deployments are supported.
 _/etc/init.d/_ scripts are not supported for now, but it can be easily added.
-For other deployment options, a ClusterManager can be implemented and plugged in. 
+For other deployment options, a ClusterManager can be implemented and plugged in.
 
 [[maven.build.commands.integration.tests.destructive]]
-==== Destructive integration / system tests
+==== Destructive integration / system tests (ChaosMonkey)
+
+HBase 0.96 introduced a tool named `ChaosMonkey`, modeled after
+link:http://techblog.netflix.com/2012/07/chaos-monkey-released-into-wild.html[same-named tool by Netflix's Chaos Monkey tool].
+ChaosMonkey simulates real-world
+faults in a running cluster by killing or disconnecting random servers, or injecting
+other failures into the environment. You can use ChaosMonkey as a stand-alone tool
+to run a policy while other tests are running. In some environments, ChaosMonkey is
+always running, in order to constantly check that high availability and fault tolerance
+are working as expected.
 
-In 0.96, a tool named `ChaosMonkey` has been introduced.
-It is modeled after the link:http://techblog.netflix.com/2012/07/chaos-monkey-released-into-wild.html[same-named tool by Netflix].
-Some of the tests use ChaosMonkey to simulate faults in the running cluster in the way of killing random servers, disconnecting servers, etc.
-ChaosMonkey can also be used as a stand-alone tool to run a (misbehaving) policy while you are running other tests. 
+ChaosMonkey defines *Actions* and *Policies*.
 
-ChaosMonkey defines Action's and Policy's.
-Actions are sequences of events.
-We have at least the following actions:
+Actions:: Actions are predefined sequences of events, such as the following:
 
 * Restart active master (sleep 5 sec)
 * Restart random regionserver (sleep 5 sec)
@@ -1215,23 +1269,17 @@ We have at least the following actions:
 * Batch restart of 50% of regionservers (sleep 5 sec)
 * Rolling restart of 100% of regionservers (sleep 5 sec)
 
-Policies on the other hand are responsible for executing the actions based on a strategy.
-The default policy is to execute a random action every minute based on predefined action weights.
-ChaosMonkey executes predefined named policies until it is stopped.
-More than one policy can be active at any time. 
-
-To run ChaosMonkey as a standalone tool deploy your HBase cluster as usual.
-ChaosMonkey uses the configuration from the bin/hbase script, thus no extra configuration needs to be done.
-You can invoke the ChaosMonkey by running:
-
-[source,bourne]
-----
-bin/hbase org.apache.hadoop.hbase.util.ChaosMonkey
-----
+Policies:: A policy is a strategy for executing one or more actions. The default policy
+executes a random action every minute based on predefined action weights.
+A given policy will be executed until ChaosMonkey is interrupted.
 
-This will output smt like: 
+Most ChaosMonkey actions are configured to have reasonable defaults, so you can run
+ChaosMonkey against an existing cluster without any additional configuration. The
+following example runs ChaosMonkey with the default configuration:
 
+[source,bash]
 ----
+$ bin/hbase org.apache.hadoop.hbase.util.ChaosMonkey
 
 12/11/19 23:21:57 INFO util.ChaosMonkey: Using ChaosMonkey Policy: class org.apache.hadoop.hbase.util.ChaosMonkey$PeriodicRandomActionPolicy, period:60000
 12/11/19 23:21:57 INFO util.ChaosMonkey: Sleeping for 26953 to add jitter
@@ -1270,31 +1318,38 @@ This will output smt like:
 12/11/19 23:24:27 INFO util.ChaosMonkey: Started region server:rs3.example.com,60020,1353367027826. Reported num of rs:6
 ----
 
-As you can see from the log, ChaosMonkey started the default PeriodicRandomActionPolicy, which is configured with all the available actions, and ran RestartActiveMaster and RestartRandomRs actions.
-ChaosMonkey tool, if run from command line, will keep on running until the process is killed. 
+The output indicates that ChaosMonkey started the default `PeriodicRandomActionPolicy`
+policy, which is configured with all the available actions. It chose to run `RestartActiveMaster` and `RestartRandomRs` actions.
+
+==== Available Policies
+HBase ships with several ChaosMonkey policies, available in the
+`hbase/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/` directory.
 
 [[chaos.monkey.properties]]
-==== Passing individual Chaos Monkey per-test Settings/Properties
+==== Configuring Individual ChaosMonkey Actions
 
-Since HBase version 1.0.0 (link:https://issues.apache.org/jira/browse/HBASE-11348[HBASE-11348]), the chaos monkeys is used to run integration tests can be configured per test run.
-Users can create a java properties file and and pass this to the chaos monkey with timing configurations.
-The properties file needs to be in the HBase classpath.
-The various properties that can be configured and their default values can be found listed in the `org.apache.hadoop.hbase.chaos.factories.MonkeyConstants`                    class.
-If any chaos monkey configuration is missing from the property file, then the default values are assumed.
-For example:
+Since HBase version 1.0.0 (link:https://issues.apache.org/jira/browse/HBASE-11348[HBASE-11348]),
+ChaosMonkey integration tests can be configured per test run.
+Create a Java properties file in the HBase classpath and pass it to ChaosMonkey using
+the `-monkeyProps` configuration flag. Configurable properties, along with their default
+values if applicable, are listed in the `org.apache.hadoop.hbase.chaos.factories.MonkeyConstants`
+class. For properties that have defaults, you can override them by including them
+in your properties file.
+
+The following example uses a properties file called <<monkey.properties,monkey.properties>>.
 
 [source,bourne]
 ----
-
-$bin/hbase org.apache.hadoop.hbase.IntegrationTestIngest -m slowDeterministic -monkeyProps monkey.properties
+$ bin/hbase org.apache.hadoop.hbase.IntegrationTestIngest -m slowDeterministic -monkeyProps monkey.properties
 ----
 
 The above command will start the integration tests and chaos monkey passing the properties file _monkey.properties_.
 Here is an example chaos monkey file:
 
+[[monkey.properties]]
+.Example ChaosMonkey Properties File
 [source]
 ----
-
 sdm.action1.period=120000
 sdm.action2.period=40000
 move.regions.sleep.time=80000
@@ -1303,6 +1358,35 @@ move.regions.sleep.time=80000
 batch.restart.rs.ratio=0.4f
 ----
 
+HBase 1.0.2 and newer adds the ability to restart HBase's underlying ZooKeeper quorum or
+HDFS nodes. To use these actions, you need to configure some new properties, which
+have no reasonable defaults because they are deployment-specific, in your ChaosMonkey
+properties file, which may be `hbase-site.xml` or a different properties file.
+
+[source,xml]
+----
+<property>
+  <name>hbase.it.clustermanager.hadoop.home</name>
+  <value>$HADOOP_HOME</value>
+</property>
+<property>
+  <name>hbase.it.clustermanager.zookeeper.home</name>
+  <value>$ZOOKEEPER_HOME</value>
+</property>
+<property>
+  <name>hbase.it.clustermanager.hbase.user</name>
+  <value>hbase</value>
+</property>
+<property>
+  <name>hbase.it.clustermanager.hadoop.hdfs.user</name>
+  <value>hdfs</value>
+</property>
+<property>
+  <name>hbase.it.clustermanager.zookeeper.user</name>
+  <value>zookeeper</value>
+</property>
+----
+
 [[developing]]
 == Developer Guidelines
 
@@ -1326,25 +1410,39 @@ NOTE: End-of-life releases are not included in this list.
 |===
 | Release
 | Release Manager
+
+| 0.94
+| Lars Hofhansl
+
 | 0.98
 | Andrew Purtell
 
 | 1.0
 | Enis Soztutar
+
+| 1.1
+| Nick Dimiduk
+
+| 1.2
+| Sean Busbey
+
+| 1.3
+| Mikhail Antonov
+
 |===
 
 [[code.standards]]
 === Code Standards
 
-See <<eclipse.code.formatting,eclipse.code.formatting>> and <<common.patch.feedback,common.patch.feedback>>. 
+See <<eclipse.code.formatting,eclipse.code.formatting>> and <<common.patch.feedback,common.patch.feedback>>.
 
 ==== Interface Classifications
 
 Interfaces are classified both by audience and by stability level.
 These labels appear at the head of a class.
-The conventions followed by HBase are inherited by its parent project, Hadoop. 
+The conventions followed by HBase are inherited by its parent project, Hadoop.
 
-The following interface classifications are commonly used: 
+The following interface classifications are commonly used:
 
 .InterfaceAudience
 `@InterfaceAudience.Public`::
@@ -1368,7 +1466,7 @@ No `@InterfaceAudience` Classification::
 .Excluding Non-Public Interfaces from API Documentation
 [NOTE]
 ====
-Only interfaces classified `@InterfaceAudience.Public` should be included in API documentation (Javadoc). Committers must add new package excludes `ExcludePackageNames` section of the _pom.xml_ for new packages which do not contain public classes. 
+Only interfaces classified `@InterfaceAudience.Public` should be included in API documentation (Javadoc). Committers must add new package excludes `ExcludePackageNames` section of the _pom.xml_ for new packages which do not contain public classes.
 ====
 
 .@InterfaceStability
@@ -1386,7 +1484,7 @@ Only interfaces classified `@InterfaceAudience.Public` should be included in API
 No `@InterfaceStability` Label::
   Public classes with no `@InterfaceStability` label are discouraged, and should be considered implicitly unstable.
 
-If you are unclear about how to mark packages, ask on the development list. 
+If you are unclear about how to mark packages, ask on the development list.
 
 [[common.patch.feedback]]
 ==== Code Formatting Conventions
@@ -1489,7 +1587,7 @@ Don't forget Javadoc!
 
 Javadoc warnings are checked during precommit.
 If the precommit tool gives you a '-1', please fix the javadoc issue.
-Your patch won't be committed if it adds such warnings. 
+Your patch won't be committed if it adds such warnings.
 
 [[common.patch.feedback.findbugs]]
 ===== Findbugs
@@ -1509,7 +1607,9 @@ value="HE_EQUALS_USE_HASHCODE",
 justification="I know what I'm doing")
 ----
 
-It is important to use the Apache-licensed version of the annotations. 
+It is important to use the Apache-licensed version of the annotations. That generally means using
+annotations in the `edu.umd.cs.findbugs.annotations` package so that we can rely on the cleanroom
+reimplementation rather than annotations in the `javax.annotations` package.
 
 [[common.patch.feedback.javadoc.defaults]]
 ===== Javadoc - Useless Defaults
@@ -1533,14 +1633,14 @@ The preference is to add something descriptive and useful.
 [[common.patch.feedback.onething]]
 ===== One Thing At A Time, Folks
 
-If you submit a patch for one thing, don't do auto-reformatting or unrelated reformatting of code on a completely different area of code. 
+If you submit a patch for one thing, don't do auto-reformatting or unrelated reformatting of code on a completely different area of code.
 
-Likewise, don't add unrelated cleanup or refactorings outside the scope of your Jira. 
+Likewise, don't add unrelated cleanup or refactorings outside the scope of your Jira.
 
 [[common.patch.feedback.tests]]
 ===== Ambigious Unit Tests
 
-Make sure that you're clear about what you are testing in your unit tests and why. 
+Make sure that you're clear about what you are testing in your unit tests and why.
 
 [[common.patch.feedback.writable]]
 ===== Implementing Writable
@@ -1548,24 +1648,38 @@ Make sure that you're clear about what you are testing in your unit tests and wh
 .Applies pre-0.96 only
 [NOTE]
 ====
-In 0.96, HBase moved to protocol buffers (protobufs). The below section on Writables applies to 0.94.x and previous, not to 0.96 and beyond. 
+In 0.96, HBase moved to protocol buffers (protobufs). The below section on Writables applies to 0.94.x and previous, not to 0.96 and beyond.
 ====
 
 Every class returned by RegionServers must implement the `Writable` interface.
-If you are creating a new class that needs to implement this interface, do not forget the default constructor. 
+If you are creating a new class that needs to implement this interface, do not forget the default constructor.
+
+==== Garbage-Collection Conserving Guidelines
+
+The following guidelines were borrowed from http://engineering.linkedin.com/performance/linkedin-feed-faster-less-jvm-garbage.
+Keep them in mind to keep preventable garbage  collection to a minimum. Have a look
+at the blog post for some great examples of how to refactor your code according to
+these guidelines.
+
+- Be careful with Iterators
+- Estimate the size of a collection when initializing
+- Defer expression evaluation
+- Compile the regex patterns in advance
+- Cache it if you can
+- String Interns are useful but dangerous
 
 [[design.invariants]]
 === Invariants
 
 We don't have many but what we have we list below.
-All are subject to challenge of course but until then, please hold to the rules of the road. 
+All are subject to challenge of course but until then, please hold to the rules of the road.
 
 [[design.invariants.zk.data]]
 ==== No permanent state in ZooKeeper
 
 ZooKeeper state should transient (treat it like memory). If ZooKeeper state is deleted, hbase should be able to recover and essentially be in the same state.
 
-* .ExceptionsThere are currently a few exceptions that we need to fix around whether a table is enabled or disabled.
+* .Exceptions: There are currently a few exceptions that we need to fix around whether a table is enabled or disabled.
 * Replication data is currently stored only in ZooKeeper.
   Deleting ZooKeeper data related to replication may cause replication to be disabled.
   Do not delete the replication tree, _/hbase/replication/_.
@@ -1579,14 +1693,14 @@ Follow progress on this issue at link:https://issues.apache.org/jira/browse/HBAS
 
 If you are developing Apache HBase, frequently it is useful to test your changes against a more-real cluster than what you find in unit tests.
 In this case, HBase can be run directly from the source in local-mode.
-All you need to do is run: 
+All you need to do is run:
 
 [source,bourne]
 ----
 ${HBASE_HOME}/bin/start-hbase.sh
 ----
 
-This will spin up a full local-cluster, just as if you had packaged up HBase and installed it on your machine. 
+This will spin up a full local-cluster, just as if you had packaged up HBase and installed it on your machine.
 
 Keep in mind that you will need to have installed HBase into your local maven repository for the in-situ cluster to work properly.
 That is, you will need to run:
@@ -1607,21 +1721,21 @@ HBase exposes metrics using the Hadoop Metrics 2 system, so adding a new metric
 Unfortunately the API of metrics2 changed from hadoop 1 to hadoop 2.
 In order to get around this a set of interfaces and implementations have to be loaded at runtime.
 To get an in-depth look at the reasoning and structure of these classes you can read the blog post located link:https://blogs.apache.org/hbase/entry/migration_to_the_new_metrics[here].
-To add a metric to an existing MBean follow the short guide below: 
+To add a metric to an existing MBean follow the short guide below:
 
 ==== Add Metric name and Function to Hadoop Compat Interface.
 
 Inside of the source interface the corresponds to where the metrics are generated (eg MetricsMasterSource for things coming from HMaster) create new static strings for metric name and description.
-Then add a new method that will be called to add new reading. 
+Then add a new method that will be called to add new reading.
 
 ==== Add the Implementation to Both Hadoop 1 and Hadoop 2 Compat modules.
 
 Inside of the implementation of the source (eg.
 MetricsMasterSourceImpl in the above example) create a new histogram, counter, gauge, or stat in the init method.
-Then in the method that was added to the interface wire up the parameter passed in to the histogram. 
+Then in the method that was added to the interface wire up the parameter passed in to the histogram.
 
 Now add tests that make sure the data is correctly exported to the metrics 2 system.
-For this the MetricsAssertHelper is provided. 
+For this the MetricsAssertHelper is provided.
 
 [[git.best.practices]]
 === Git Best Practices
@@ -1661,42 +1775,43 @@ It provides a nice overview that applies equally to the Apache HBase Project.
 [[submitting.patches.create]]
 ==== Create Patch
 
-The script _dev-support/make_patch.sh_ has been provided to help you adhere to patch-creation guidelines.
-The script has the following syntax: 
+Use _dev-support/submit-patch.py_ to create patches and optionally, upload to jira and update
+reviews on Review Board. Patch name is formatted as (JIRA).(branch name).(patch number).patch to
+follow Yetus' naming rules. Use `-h` flag to know detailed usage information. Most useful options
+are:
 
-----
-$ make_patch.sh [-a] [-p <patch_dir>]
-----
+. `-b BRANCH, --branch BRANCH` : Specify base branch for generating the diff. If not specified, tracking branch is used. If there is no tracking branch, error will be thrown.
+. `-jid JIRA_ID, --jira-id JIRA_ID` : Jira id of the issue. If set, we deduce next patch version from attachments in the jira and also upload the new patch. Script will ask for jira username/password for authentication. If not set, patch is named <branch>.patch.
+
+The script builds a new patch, and uses REST API to upload it to the jira (if --jira-id is
+specified) and update the review on ReviewBoard (if --skip-review-board not specified).
+Remote links in the jira are used to figure out if a review request already exists. If no review
+request is present, then creates a new one and populates all required fields using jira summary,
+patch description, etc. Also adds this review's link to the jira.
+
+Authentication::
+Since attaching patches on JIRA and creating/changing review request on ReviewBoard requires a
+logged in user, the script will prompt you for username and password. To avoid the hassle every
+time, set up `~/.apache-creds` with login details and encrypt it by following the steps in footer
+of script's help message.
 
-. If you do not pass a `patch_dir`, the script defaults to _~/patches/_.
-  If the `patch_dir` does not exist, it is created.
-. By default, if an existing patch exists with the JIRA ID, the version of the new patch is incremented (_HBASE-XXXX-v3.patch_). If the `-a`                            option is passed, the version is not incremented, but the suffix `-addendum` is added (_HBASE-XXXX-v2-addendum.patch_). A second addendum to a given version is not supported.
-. Detects whether you have more than one local commit on your branch.
-  If you do, the script offers you the chance to run +git rebase
-  -i+ to squash the changes into a single commit so that it can use +git format-patch+.
-  If you decline, the script uses +git diff+ instead.
-  The patch is saved in a configurable directory and is ready to be attached to your JIRA.
+Python dependencies::
+To install required python dependencies, execute
+`pip install -r dev-support/python-requirements.txt` from the master branch.
 
-* .Patching WorkflowAlways patch against the master branch first, even if you want to patch in another branch.
+.Patching Workflow
+
+* Always patch against the master branch first, even if you want to patch in another branch.
   HBase committers always apply patches first to the master branch, and backport if necessary.
 * Submit one single patch for a fix.
   If necessary, squash local commits to merge local commits into a single one first.
   See this link:http://stackoverflow.com/questions/5308816/how-to-use-git-merge-squash[Stack Overflow question] for more information about squashing commits.
-* The patch should have the JIRA ID in the name.
-  If you are generating from a branch, include the target branch in the filename.
-  A common naming scheme for patches is:
-+
-----
-HBASE-XXXX.patch
-----
+* Patch name should be as follows to adhere to Yetus' naming convention.
 +
 ----
-HBASE-XXXX-0.90.patch     # to denote that the patch is against branch 0.90
-----
-+
-----
-HBASE-XXXX-v3.patch       # to denote that this is the third version of the patch
+(JIRA).(branch name).(patch number).patch
 ----
+For eg. HBASE-11625.master.001.patch, HBASE-XXXXX.branch-1.2.0005.patch, etc.
 
 * To submit a patch, first create it using one of the methods in <<patching.methods,patching.methods>>.
   Next, attach the patch to the JIRA (one patch for the whole fix), using the  dialog.
@@ -1710,20 +1825,23 @@ Please understand that not every patch may get committed, and that feedback will
 * If you need to revise your patch, leave the previous patch file(s) attached to the JIRA, and upload the new one, following the naming conventions in <<submitting.patches.create,submitting.patches.create>>.
   Cancel the Patch Available flag and then re-trigger it, by toggling the btn:[Patch Available] button in JIRA.
   JIRA sorts attached files by the time they were attached, and has no problem with multiple attachments with the same name.
-  However, at times it is easier to refer to different version of a patch if you add `-vX`, where the [replaceable]_X_ is the version (starting with 2).
-* If you need to submit your patch against multiple branches, rather than just master, name each version of the patch with the branch it is for, following the naming conventions in <<submitting.patches.create,submitting.patches.create>>.
+  However, at times it is easier to increment patch number in the patch name.
 
-.Methods to Create PatchesEclipse::
+[[patching.methods]]
+.Methods to Create Patches
+Eclipse::
   Select the  menu item.
 
 Git::
-  `git format-patch` is preferred because it preserves commit messages.
+  `git format-patch` is preferred:
+     - It preserves the committer and commit message.
+     - It handles binary files by default, whereas `git diff` ignores them unless
+     you use the `--binary` option.
   Use `git rebase -i` first, to combine (squash) smaller commits into a single larger one.
 
 Subversion::
-
-Make sure you review <<eclipse.code.formatting,eclipse.code.formatting>> and <<common.patch.feedback,common.patch.feedback>> for code style.
-If your patch was generated incorrectly or your code does not adhere to the code formatting guidelines, you may be asked to redo some work.
+  Make sure you review <<eclipse.code.formatting,eclipse.code.formatting>> and <<common.patch.feedback,common.patch.feedback>> for code style.
+  If your patch was generated incorrectly or your code does not adhere to the code formatting guidelines, you may be asked to redo some work.
 
 [[submitting.patches.tests]]
 ==== Unit Tests
@@ -1735,21 +1853,22 @@ Also, see <<mockito,mockito>>.
 
 If you are creating a new unit test class, notice how other unit test classes have classification/sizing annotations at the top and a static method on the end.
 Be sure to include these in any new unit test files you generate.
-See <<hbase.tests,hbase.tests>> for more on how the annotations work. 
+See <<hbase.tests,hbase.tests>> for more on how the annotations work.
 
 ==== Integration Tests
 
 Significant new features should provide an integration test in addition to unit tests, suitable for exercising the new feature at different points in its configuration space.
 
+[[reviewboard]]
 ==== ReviewBoard
 
-Patches larger than one screen, or patches that will be tricky to review, should go through link:http://reviews.apache.org[ReviewBoard]. 
+Patches larger than one screen, or patches that will be tricky to review, should go through link:http://reviews.apache.org[ReviewBoard].
 
 .Procedure: Use ReviewBoard
 . Register for an account if you don't already have one.
   It does not use the credentials from link:http://issues.apache.org[issues.apache.org].
   Log in.
-. Click [label]#New Review Request#. 
+. Click [label]#New Review Request#.
 . Choose the `hbase-git` repository.
   Click Choose File to select the diff and optionally a parent diff.
   Click btn:[Create
@@ -1765,39 +1884,39 @@ Patches larger than one screen, or patches that will be tricky to review, should
 . To cancel the request, click .
 
 For more information on how to use ReviewBoard, see link:http://www.reviewboard.org/docs/manual/1.5/[the ReviewBoard
-                        documentation]. 
+                        documentation].
 
 ==== Guide for HBase Committers
 
 ===== New committers
 
-New committers are encouraged to first read Apache's generic committer documentation: 
+New committers are encouraged to first read Apache's generic committer documentation:
 
-* link:http://www.apache.org/dev/new-committers-guide.html[Apache New Committer Guide]                            
-* link:http://www.apache.org/dev/committers.html[Apache Committer FAQ]                            
+* link:http://www.apache.org/dev/new-committers-guide.html[Apache New Committer Guide]
+* link:http://www.apache.org/dev/committers.html[Apache Committer FAQ]
 
 ===== Review
 
 HBase committers should, as often as possible, attempt to review patches submitted by others.
 Ideally every submitted patch will get reviewed by a committer _within a few days_.
-If a committer reviews a patch they have not authored, and believe it to be of sufficient quality, then they can commit the patch, otherwise the patch should be cancelled with a clear explanation for why it was rejected. 
+If a committer reviews a patch they have not authored, and believe it to be of sufficient quality, then they can commit the patch, otherwise the patch should be cancelled with a clear explanation for why it was rejected.
 
 The list of submitted patches is in the link:https://issues.apache.org/jira/secure/IssueNavigator.jspa?mode=hide&requestId=12312392[HBase Review Queue], which is ordered by time of last modification.
-Committers should scan the list from top to bottom, looking for patches that they feel qualified to review and possibly commit. 
+Committers should scan the list from top to bottom, looking for patches that they feel qualified to review and possibly commit.
 
 For non-trivial changes, it is required to get another committer to review your own patches before commit.
-Use the btn:[Submit Patch]                        button in JIRA, just like other contributors, and then wait for a `+1` response from another committer before committing. 
+Use the btn:[Submit Patch]                        button in JIRA, just like other contributors, and then wait for a `+1` response from another committer before committing.
 
 ===== Reject
 
 Patches which do not adhere to the guidelines in link:https://wiki.apache.org/hadoop/Hbase/HowToCommit/hadoop/Hbase/HowToContribute#[HowToContribute] and to the link:https://wiki.apache.org/hadoop/Hbase/HowToCommit/hadoop/CodeReviewChecklist#[code review checklist] should be rejected.
 Committers should always be polite to contributors and try to instruct and encourage them to contribute better patches.
-If a committer wishes to improve an unacceptable patch, then it should first be rejected, and a new patch should be attached by the committer for review. 
+If a committer wishes to improve an unacceptable patch, then it should first be rejected, and a new patch should be attached by the committer for review.
 
 [[committing.patches]]
 ===== Commit
 
-Committers commit patches to the Apache HBase GIT repository. 
+Committers commit patches to the Apache HBase GIT repository.
 
 .Before you commit!!!!
 [NOTE]
@@ -1805,13 +1924,13 @@ Committers commit patches to the Apache HBase GIT repository.
 Make sure your local configuration is correct, especially your identity and email.
 Examine the output of the +$ git config
                                 --list+ command and be sure it is correct.
-See this GitHub article, link:https://help.github.com/articles/set-up-git[Set Up Git] if you need pointers. 
+See this GitHub article, link:https://help.github.com/articles/set-up-git[Set Up Git] if you need pointers.
 ====
 
-When you commit a patch, please: 
+When you commit a patch, please:
 
 . Include the Jira issue id in the commit message, along with a short description of the change and the name of the contributor if it is not you.
-  Be sure to get the issue ID right, as this causes Jira to link to the change in Git (use the issue's "All" tab to see these). 
+  Be sure to get the issue ID right, as this causes Jira to link to the change in Git (use the issue's "All" tab to see these).
 . Commit the patch to a new branch based off master or other intended branch.
   It's a good idea to call this branch by the JIRA ID.
   Then check out the relevant target branch where you want to commit, make sure your local branch has all remote changes, by doing a +git pull --rebase+ or another similar command, cherry-pick the change into each relevant branch (such as master), and do +git push <remote-server>
@@ -1822,9 +1941,9 @@ If the push fails for any reason, fix the problem or ask for help.
 Do not do a +git push --force+.
 +
 Before you can commit a patch, you need to determine how the patch was created.
-The instructions and preferences around the way to create patches have changed, and there will be a transition periond.
+The instructions and preferences around the way to create patches have changed, and there will be a transition period.
 +
-* .Determine How a Patch Was CreatedIf the first few lines of the patch look like the headers of an email, with a From, Date, and Subject, it was created using +git format-patch+.
+* .Determine How a Patch Was Created: If the first few lines of the patch look like the headers of an email, with a From, Date, and Subject, it was created using +git format-patch+.
   This is the preference, because you can reuse the submitter's commit message.
   If the commit message is not appropriate, you can still use the commit, then run the command +git
   rebase -i origin/master+, and squash and reword as appropriate.
@@ -1834,13 +1953,13 @@ The instructions and preferences around the way to create patches have changed,
   This is the indication that the patch was not created with `--no-prefix`.
 +
 ----
-diff --git a/src/main/docbkx/developer.xml b/src/main/docbkx/developer.xml
+diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc
 ----
 
 * If the first line of the patch looks similar to the following (without the `a` and `b`), the patch was created with +git diff --no-prefix+ and you need to add `-p0` to the +git apply+                                        command below.
 +
 ----
-diff --git src/main/docbkx/developer.xml src/main/docbkx/developer.xml
+diff --git src/main/asciidoc/_chapters/developer.adoc src/main/asciidoc/_chapters/developer.adoc
 ----
 
 +
@@ -1851,7 +1970,7 @@ The only command that actually writes anything to the remote repository is +git
 The extra +git
                                         pull+ commands are usually redundant, but better safe than sorry.
 
-The first example shows how to apply a patch that was generated with +git format-patch+ and apply it to the `master` and `branch-1` branches. 
+The first example shows how to apply a patch that was generated with +git format-patch+ and apply it to the `master` and `branch-1` branches.
 
 The directive to use +git format-patch+                                    rather than +git diff+, and not to use `--no-prefix`, is a new one.
 See the second example for how to apply a patch created with +git
@@ -1879,7 +1998,7 @@ This example shows how to commit a patch that was created using +git diff+ witho
 If the patch was created with `--no-prefix`, add `-p0` to the +git apply+ command.
 
 ----
-$ git apply ~/Downloads/HBASE-XXXX-v2.patch 
+$ git apply ~/Downloads/HBASE-XXXX-v2.patch
 $ git commit -m "HBASE-XXXX Really Good Code Fix (Joe Schmo)" -a # This extra step is needed for patches created with 'git diff'
 $ git checkout master
 $ git pull --rebase
@@ -1898,7 +2017,7 @@ $ git branch -D HBASE-XXXX
 ====
 
 . Resolve the issue as fixed, thanking the contributor.
-  Always set the "Fix Version" at this point, but please only set a single fix version for each branch where the change was committed, the earliest release in that branch in which the change will appear. 
+  Always set the "Fix Version" at this point, but please only set a single fix version for each branch where the change was committed, the earliest release in that branch in which the change will appear.
 
 ====== Commit Message Format
 
@@ -1918,30 +2037,30 @@ If the contributor used +git format-patch+ to generate the patch, their commit m
 [[committer.amending.author]]
 ====== Add Amending-Author when a conflict cherrypick backporting
 
-We've established the practice of committing to trunk and then cherry picking back to branches whenever possible.
+We've established the practice of committing to master and then cherry picking back to branches whenever possible.
 When there is a minor conflict we can fix it up and just proceed with the commit.
 The resulting commit retains the original author.
 When the amending author is different from the original committer, add notice of this at the end of the commit message as: `Amending-Author: Author
                                 <committer&apache>` See discussion at link:http://search-hadoop.com/m/DHED4wHGYS[HBase, mail # dev
                                 - [DISCUSSION] Best practice when amending commits cherry picked
-                                from master to branch]. 
+                                from master to branch].
 
 [[committer.tests]]
-====== Committers are responsible for making sure commits do not break thebuild or tests
+====== Committers are responsible for making sure commits do not break the build or tests
 
 If a committer commits a patch, it is their responsibility to make sure it passes the test suite.
 It is helpful if contributors keep an eye out that their patch does not break the hbase build and/or tests, but ultimately, a contributor cannot be expected to be aware of all the particular vagaries and interconnections that occur in a project like HBase.
-A committer should. 
+A committer should.
 
 [[git.patch.flow]]
 ====== Patching Etiquette
 
 In the thread link:http://search-hadoop.com/m/DHED4EiwOz[HBase, mail # dev - ANNOUNCEMENT: Git Migration In Progress (WAS =>
-                                Re: Git Migration)], it was agreed on the following patch flow 
+                                Re: Git Migration)], it was agreed on the following patch flow
 
-. Develop and commit the patch against trunk/master first.
+. Develop and commit the patch against master first.
 . Try to cherry-pick the patch when backporting if possible.
-. If this does not work, manually commit the patch to the branch.                        
+. If this does not work, manually commit the patch to the branch.
 
 ====== Merge Commits
 
@@ -1954,11 +2073,11 @@ See <<appendix_contributing_to_documentation,appendix contributing to documentat
 ==== Dialog
 
 Committers should hang out in the #hbase room on irc.freen

<TRUNCATED>

[7/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 0aac442..cfdd638 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -41,7 +41,8 @@ Technically speaking, HBase is really more a "Data Store" than "Data Base" becau
 However, HBase has many features which supports both linear and modular scaling.
 HBase clusters expand by adding RegionServers that are hosted on commodity class servers.
 If a cluster expands from 10 to 20 RegionServers, for example, it doubles both in terms of storage and as well as processing capacity.
-RDBMS can scale well, but only up to a point - specifically, the size of a single database server - and for the best performance requires specialized hardware and storage devices.
+An RDBMS can scale well, but only up to a point - specifically, the size of a single database
+server - and for the best performance requires specialized hardware and storage devices.
 HBase features of note are:
 
 * Strongly consistent reads/writes:  HBase is not an "eventually consistent" DataStore.
@@ -138,7 +139,10 @@ A region with an empty start key is the first region in a table.
 If a region has both an empty start and an empty end key, it is the only region in the table
 ====
 
-In the (hopefully unlikely) event that programmatic processing of catalog metadata is required, see the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/util/Writables.html#getHRegionInfo%28byte[]%29[Writables] utility.
+In the (hopefully unlikely) event that programmatic processing of catalog metadata
+is required, see the
++++<a href="http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/Writables.html#getHRegionInfo%28byte%5B%5D%29">Writables</a>+++
+utility.
 
 [[arch.catalog.startup]]
 === Startup Sequencing
@@ -169,7 +173,7 @@ The API changed in HBase 1.0. For connection configuration information, see <<cl
 
 ==== API as of HBase 1.0.0
 
-Its been cleaned up and users are returned Interfaces to work against rather than particular types.
+It's been cleaned up and users are returned Interfaces to work against rather than particular types.
 In HBase 1.0, obtain a `Connection` object from `ConnectionFactory` and thereafter, get from it instances of `Table`, `Admin`, and `RegionLocator` on an as-need basis.
 When done, close the obtained instances.
 Finally, be sure to cleanup your `Connection` instance before exiting.
@@ -235,11 +239,11 @@ Please use link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/C
 [[client.writebuffer]]
 === WriteBuffer and Batch Methods
 
-In HBase 1.0 and later, link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] is deprecated in favor of link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table]. `Table` does not use autoflush. To do buffered writes, use the BufferedMutator class.
+In HBase 1.0 and later, link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable] is deprecated in favor of link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html[Table]. `Table` does not use autoflush. To do buffered writes, use the BufferedMutator class.
 
 Before a `Table` or `HTable` instance is discarded, invoke either `close()` or `flushCommits()`, so `Put`s will not be lost.
 
-For additional information on write durability, review the link:../acid-semantics.html[ACID semantics] page.
+For additional information on write durability, review the link:/acid-semantics.html[ACID semantics] page.
 
 For fine-grained control of batching of ``Put``s or ``Delete``s, see the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch%28java.util.List%29[batch] methods on Table.
 
@@ -292,7 +296,11 @@ scan.setFilter(list);
 [[client.filter.cv.scvf]]
 ==== SingleColumnValueFilter
 
-link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html[SingleColumnValueFilter] can be used to test column values for equivalence (`CompareOp.EQUAL`), inequality (`CompareOp.NOT_EQUAL`), or ranges (e.g., `CompareOp.GREATER`). The following is example of testing equivalence a column to a String value "my value"...
+A SingleColumnValueFilter (see:
+http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.html)
+can be used to test column values for equivalence (`CompareOp.EQUAL`),
+inequality (`CompareOp.NOT_EQUAL`), or ranges (e.g., `CompareOp.GREATER`). The following is an
+example of testing equivalence of a column to a String value "my value"...
 
 [source,java]
 ----
@@ -493,6 +501,7 @@ It is generally a better idea to use the startRow/stopRow methods on Scan for ro
 This is primarily used for rowcount jobs.
 See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.html[FirstKeyOnlyFilter].
 
+[[architecture.master]]
 == Master
 
 `HMaster` is the implementation of the Master Server.
@@ -691,7 +700,8 @@ Here are others that you may have to take into account:
 
 Catalog Tables::
   The `-ROOT-` (prior to HBase 0.96, see <<arch.catalog.root,arch.catalog.root>>) and `hbase:meta` tables are forced into the block cache and have the in-memory priority which means that they are harder to evict.
-  The former never uses more than a few hundreds bytes while the latter can occupy a few MBs (depending on the number of regions).
+  The former never uses more than a few hundred bytes while the latter can occupy a few MBs
+  (depending on the number of regions).
 
 HFiles Indexes::
   An _HFile_ is the file format that HBase uses to store data in HDFS.
@@ -759,7 +769,7 @@ When we go to look for a cached block, we look first in L1 and if none found, th
 Let us call this deploy format, _Raw L1+L2_.
 
 Other BucketCache configs include: specifying a location to persist cache to across restarts, how many threads to use writing the cache, etc.
-See the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig.html] class for configuration options and descriptions.
+See the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html[CacheConfig.html] class for configuration options and descriptions.
 
 
 
@@ -875,7 +885,10 @@ image::region_split_process.png[Region Split Process]
 . The Master learns about this znode, since it has a watcher for the parent `region-in-transition` znode.
 . The RegionServer creates a sub-directory named `.splits` under the parent\u2019s `region` directory in HDFS.
 . The RegionServer closes the parent region and marks the region as offline in its local data structures. *THE SPLITTING REGION IS NOW OFFLINE.* At this point, client requests coming to the parent region will throw `NotServingRegionException`. The client will retry with some backoff. The closing region is flushed.
-. The  RegionServer creates region directories under the `.splits` directory, for daughter regions A and B, and creates necessary data structures. Then it splits the store files, in the sense that it creates two link:http://www.google.com/url?q=http%3A%2F%2Fhbase.apache.org%2Fapidocs%2Forg%2Fapache%2Fhadoop%2Fhbase%2Fio%2FReference.html&sa=D&sntz=1&usg=AFQjCNEkCbADZ3CgKHTtGYI8bJVwp663CA[Reference] files per store file in the parent region. Those reference files will point to the parent regions'files.
+. The RegionServer creates region directories under the `.splits` directory, for daughter
+regions A and B, and creates necessary data structures. Then it splits the store files,
+in the sense that it creates two Reference files per store file in the parent region.
+Those reference files will point to the parent region's files.
 . The RegionServer creates the actual region directory in HDFS, and moves the reference files for each daughter.
 . The RegionServer sends a `Put` request to the `.META.` table, to set the parent as offline in the `.META.` table and add information about daughter regions. At this point, there won\u2019t be individual entries in `.META.` for the daughters. Clients will see that the parent region is split if they scan `.META.`, but won\u2019t know about the daughters until they appear in `.META.`. Also, if this `Put` to `.META`. succeeds, the parent will be effectively split. If the RegionServer fails before this RPC succeeds, Master and the next Region Server opening the region will clean dirty state about the region split. After the `.META.` update, though, the region split will be rolled-forward by Master.
 . The RegionServer opens daughters A and B in parallel.
@@ -928,7 +941,7 @@ To configure MultiWAL for a RegionServer, set the value of the property `hbase.w
 </property>
 ----
 
-Restart the RegionServer for the changes to take effect. 
+Restart the RegionServer for the changes to take effect.
 
 To disable MultiWAL for a RegionServer, unset the property and restart the RegionServer.
 
@@ -1005,7 +1018,8 @@ If you set the `hbase.hlog.split.skip.errors` option to `true`, errors are treat
 * Processing of the WAL will continue
 
 If the `hbase.hlog.split.skip.errors` option is set to `false`, the default, the exception will be propagated and the split will be logged as failed.
-See link:https://issues.apache.org/jira/browse/HBASE-2958[HBASE-2958 When hbase.hlog.split.skip.errors is set to false, we fail the split but thats it].
+See link:https://issues.apache.org/jira/browse/HBASE-2958[HBASE-2958 When
+hbase.hlog.split.skip.errors is set to false, we fail the split but that's it].
 We need to do more than just fail split if this flag is set.
 
 ====== How EOFExceptions are treated when splitting a crashed RegionServer's WALs
@@ -1016,17 +1030,9 @@ For background, see link:https://issues.apache.org/jira/browse/HBASE-2643[HBASE-
 
 ===== Performance Improvements during Log Splitting
 
-WAL log splitting and recovery can be resource intensive and take a long time, depending on the number of RegionServers involved in the crash and the size of the regions. <<distributed.log.splitting>> and <<distributed.log.replay>> were developed to improve performance during log splitting.
+WAL log splitting and recovery can be resource intensive and take a long time, depending on the number of RegionServers involved in the crash and the size of the regions. <<distributed.log.splitting>> was developed to improve performance during log splitting.
 
 [[distributed.log.splitting]]
-====== Distributed Log Splitting
-
-_Distributed Log Splitting_ was added in HBase version 0.92 (link:https://issues.apache.org/jira/browse/HBASE-1364[HBASE-1364]) by Prakash Khemani from Facebook.
-It reduces the time to complete log splitting dramatically, improving the availability of regions and tables.
-For example, recovering a crashed cluster took around 9 hours with single-threaded log splitting, but only about six minutes with distributed log splitting.
-
-The information in this section is sourced from Jimmy Xiang's blog post at http://blog.cloudera.com/blog/2012/07/hbase-log-splitting/.
-
 .Enabling or Disabling Distributed Log Splitting
 
 Distributed log processing is enabled by default since HBase 0.92.
@@ -1123,7 +1129,8 @@ Based on the state of the task whose data is changed, the split log manager does
 Each RegionServer runs a daemon thread called the _split log worker_, which does the work to split the logs.
 The daemon thread starts when the RegionServer starts, and registers itself to watch HBase znodes.
 If any splitlog znode children change, it notifies a sleeping worker thread to wake up and grab more tasks.
-If if a worker's current task's node data is changed, the worker checks to see if the task has been taken by another worker.
+If a worker's current task's node data is changed,
+the worker checks to see if the task has been taken by another worker.
 If so, the worker thread stops work on the current task.
 +
 The worker monitors the splitlog znode constantly.
@@ -1133,7 +1140,7 @@ At this point, the split log worker scans for another unclaimed task.
 +
 .How the Split Log Worker Approaches a Task
 * It queries the task state and only takes action if the task is in `TASK_UNASSIGNED `state.
-* If the task is is in `TASK_UNASSIGNED` state, the worker attempts to set the state to `TASK_OWNED` by itself.
+* If the task is in `TASK_UNASSIGNED` state, the worker attempts to set the state to `TASK_OWNED` by itself.
   If it fails to set the state, another worker will try to grab it.
   The split log manager will also ask all workers to rescan later if the task remains unassigned.
 * If the worker succeeds in taking ownership of the task, it tries to get the task state again to make sure it really gets it asynchronously.
@@ -1141,7 +1148,7 @@ At this point, the split log worker scans for another unclaimed task.
 ** Get the HBase root folder, create a temp folder under the root, and split the log file to the temp folder.
 ** If the split was successful, the task executor sets the task to state `TASK_DONE`.
 ** If the worker catches an unexpected IOException, the task is set to state `TASK_ERR`.
-** If the worker is shutting down, set the the task to state `TASK_RESIGNED`.
+** If the worker is shutting down, set the task to state `TASK_RESIGNED`.
 ** If the task is taken by another worker, just log it.
 
 
@@ -1332,7 +1339,7 @@ image::region_states.png[]
 . Before assigning a region, the master moves the region to `OFFLINE` state automatically if it is in `CLOSED` state.
 . When a RegionServer is about to split a region, it notifies the master.
   The master moves the region to be split from `OPEN` to `SPLITTING` state and add the two new regions to be created to the RegionServer.
-  These two regions are in `SPLITING_NEW` state initially.
+  These two regions are in `SPLITTING_NEW` state initially.
 . After notifying the master, the RegionServer starts to split the region.
   Once past the point of no return, the RegionServer notifies the master again so the master can update the `hbase:meta` table.
   However, the master does not update the region states until it is notified by the server that the split is done.
@@ -1377,8 +1384,10 @@ The RegionServer splits a region, offlines the split region and then adds the da
 See <<disable.splitting>> for how to manually manage splits (and for why you might do this).
 
 ==== Custom Split Policies
-ou can override the default split policy using a custom link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy](HBase 0.94+). Typically a custom split policy should extend
-HBase's default split policy: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html[IncreasingToUpperBoundRegionSplitPolicy].
+You can override the default split policy using a custom
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy](HBase 0.94+).
+Typically a custom split policy should extend HBase's default split policy:
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html[IncreasingToUpperBoundRegionSplitPolicy].
 
 The policy can set globally through the HBase configuration or on a per-table
 basis.
@@ -1397,17 +1406,19 @@ basis.
 HTableDescriptor tableDesc = new HTableDescriptor("test");
 tableDesc.setValue(HTableDescriptor.SPLIT_POLICY, ConstantSizeRegionSplitPolicy.class.getName());
 tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes("cf1")));
-admin.createTable(tableDesc);              
+admin.createTable(tableDesc);
 ----
 
 [source]
 .Configuring the Split Policy On a Table Using HBase Shell
 ----
-hbase> create 'test', {METHOD => 'table_att', CONFIG => {'SPLIT_POLICY' => 'org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy'}},
-{NAME => 'cf1'}
+hbase> create 'test', {METADATA => {'SPLIT_POLICY' => 'org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy'}},{NAME => 'cf1'}
 ----
 
-The default split policy can be overwritten using a custom link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy(HBase 0.94+)]. Typically a custom split policy should extend HBase's default split policy: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html[ConstantSizeRegionSplitPolicy].
+The default split policy can be overwritten using a custom
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.html[RegionSplitPolicy(HBase 0.94+)].
+Typically a custom split policy should extend HBase's default split policy:
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html[ConstantSizeRegionSplitPolicy].
 
 The policy can be set globally through the HBaseConfiguration used or on a per table basis:
 [source,java]
@@ -1416,6 +1427,8 @@ HTableDescriptor myHtd = ...;
 myHtd.setValue(HTableDescriptor.SPLIT_POLICY, MyCustomSplitPolicy.class.getName());
 ----
 
+NOTE: The `DisabledRegionSplitPolicy` policy blocks manual region splitting.
+
 [[manual_region_splitting_decisions]]
 === Manual Region Splitting
 
@@ -1435,6 +1448,8 @@ There may be other valid reasons, but the need to manually split your table migh
 
 See <<disable.splitting>> for a discussion about the dangers and possible benefits of managing splitting completely manually.
 
+NOTE: The `DisabledRegionSplitPolicy` policy blocks manual region splitting.
+
 ==== Determining Split Points
 
 The goal of splitting your table manually is to improve the chances of balancing the load across the cluster in situations where good rowkey design alone won't get you there.
@@ -1450,9 +1465,15 @@ Using a Custom Algorithm::
   The RegionSplitter tool is provided with HBase, and uses a _SplitAlgorithm_ to determine split points for you.
   As parameters, you give it the algorithm, desired number of regions, and column families.
   It includes two split algorithms.
-  The first is the `link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/util/RegionSplitter.HexStringSplit.html[HexStringSplit]` algorithm, which assumes the row keys are hexadecimal strings.
-  The second, `link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/util/RegionSplitter.UniformSplit.html[UniformSplit]`, assumes the row keys are random byte arrays.
-  You will probably need to develop your own `link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/util/RegionSplitter.SplitAlgorithm.html[SplitAlgorithm]`, using the provided ones as models.
+  The first is the
+  `link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.HexStringSplit.html[HexStringSplit]`
+  algorithm, which assumes the row keys are hexadecimal strings.
+  The second,
+  `link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.UniformSplit.html[UniformSplit]`,
+  assumes the row keys are random byte arrays.
+  You will probably need to develop your own
+  `link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/util/RegionSplitter.SplitAlgorithm.html[SplitAlgorithm]`,
+  using the provided ones as models.
 
 === Online Region Merges
 
@@ -1470,6 +1491,7 @@ It's an asynchronous operation and call returns immediately without waiting merg
 Passing `true` as the optional third parameter will force a merge. Normally only adjacent regions can be merged.
 The `force` parameter overrides this behaviour and is for expert use only.
 
+[[store]]
 === Store
 
 A Store hosts a MemStore and 0 or more StoreFiles (HFiles). A Store corresponds to a column family for a table for a given region.
@@ -1489,13 +1511,26 @@ Note that when the flush happens, MemStores that belong to the same region will
 A MemStore flush can be triggered under any of the conditions listed below.
 The minimum flush unit is per region, not at individual MemStore level.
 
-. When a MemStore reaches the size specified by `hbase.hregion.memstore.flush.size`, all MemStores that belong to its region will be flushed out to disk.
-. When the overall MemStore usage reaches the value specified by `hbase.regionserver.global.memstore.upperLimit`, MemStores from various regions will be flushed out to disk to reduce overall MemStore usage in a RegionServer.
-  The flush order is based on the descending order of a region's MemStore usage.
-  Regions will have their MemStores flushed until the overall MemStore usage drops to or slightly below `hbase.regionserver.global.memstore.lowerLimit`.
-. When the number of WAL per region server reaches the value specified in `hbase.regionserver.max.logs`, MemStores from various regions will be flushed out to disk to reduce WAL count.
-  The flush order is based on time.
-  Regions with the oldest MemStores are flushed first until WAL count drops below `hbase.regionserver.max.logs`.
+. When a MemStore reaches the size specified by `hbase.hregion.memstore.flush.size`,
+  all MemStores that belong to its region will be flushed out to disk.
+
+. When the overall MemStore usage reaches the value specified by
+  `hbase.regionserver.global.memstore.upperLimit`, MemStores from various regions
+  will be flushed out to disk to reduce overall MemStore usage in a RegionServer.
++
+The flush order is based on the descending order of a region's MemStore usage.
++
+Regions will have their MemStores flushed until the overall MemStore usage drops
+to or slightly below `hbase.regionserver.global.memstore.lowerLimit`.
+
+. When the number of WAL log entries in a given region server's WAL reaches the
+  value specified in `hbase.regionserver.max.logs`, MemStores from various regions
+  will be flushed out to disk to reduce the number of logs in the WAL.
++
+The flush order is based on time.
++
+Regions with the oldest MemStores are flushed first until WAL count drops below
+`hbase.regionserver.max.logs`.
 
 [[hregion.scans]]
 ==== Scans
@@ -1519,6 +1554,7 @@ Matteo Bertozzi has also put up a helpful description, link:http://th30z.blogspo
 For more information, see the link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/io/hfile/HFile.html[HFile source code].
 Also see <<hfilev2>> for information about the HFile v2 format that was included in 0.92.
 
+[[hfile_tool]]
 ===== HFile Tool
 
 To view a textualized version of HFile content, you can use the `org.apache.hadoop.hbase.io.hfile.HFile` tool.
@@ -1552,6 +1588,7 @@ For more information on compression, see <<compression>>.
 
 For more information on blocks, see the link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/io/hfile/HFileBlock.html[HFileBlock source code].
 
+[[keyvalue]]
 ==== KeyValue
 
 The KeyValue class is the heart of data storage in HBase.
@@ -1611,6 +1648,7 @@ Key portion for Put #2:
 It is critical to understand that the rowkey, ColumnFamily, and column (aka columnqualifier) are embedded within the KeyValue instance.
 The longer these identifiers are, the bigger the KeyValue is.
 
+[[compaction]]
 ==== Compaction
 
 .Ambiguous Terminology
@@ -1636,6 +1674,7 @@ The end result of a _major compaction_ is a single StoreFile per Store.
 Major compactions also process delete markers and max versions.
 See <<compaction.and.deletes>> and <<compaction.and.versions>> for information on how deletes and versions are handled in relation to compactions.
 
+[[compaction.and.deletes]]
 .Compaction and Deletions
 When an explicit deletion occurs in HBase, the data is not actually deleted.
 Instead, a _tombstone_ marker is written.
@@ -1644,6 +1683,7 @@ During a major compaction, the data is actually deleted, and the tombstone marke
 If the deletion happens because of an expired TTL, no tombstone is created.
 Instead, the expired data is filtered out and is not written back to the compacted StoreFile.
 
+[[compaction.and.versions]]
 .Compaction and Versions
 When you create a Column Family, you can specify the maximum number of versions to keep, by specifying `HColumnDescriptor.setMaxVersions(int versions)`.
 The default value is `3`.
@@ -1796,60 +1836,116 @@ This list is not exhaustive.
 To tune these parameters from the defaults, edit the _hbase-default.xml_ file.
 For a full list of all configuration parameters available, see <<config.files,config.files>>
 
-[cols="1,1a,1", options="header"]
-|===
-| Parameter
-| Description
-| Default
-
-|`hbase.hstore.compaction.min`
-| The minimum number of StoreFiles which must be eligible for compaction before compaction can run. The goal of tuning `hbase.hstore.compaction.min` is to avoid ending up with too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles in a Store, and this is probably not appropriate. If you set this value too high, all the other values will need to be adjusted accordingly. For most cases, the default value is appropriate. In previous versions of HBase, the parameter hbase.hstore.compaction.min was called `hbase.hstore.compactionThreshold`.
-|3
-
-|`hbase.hstore.compaction.max`
-| The maximum number of StoreFiles which will be selected for a single minor compaction, regardless of the number of eligible StoreFiles. Effectively, the value of hbase.hstore.compaction.max controls the length of time it takes a single compaction to complete. Setting it larger means that more StoreFiles are included in a compaction. For most cases, the default value is appropriate.
-|10
-
-|`hbase.hstore.compaction.min.size`
-| A StoreFile smaller than this size will always be eligible for minor compaction. StoreFiles this size or larger are evaluated by `hbase.hstore.compaction.ratio` to determine if they are eligible. Because this limit represents the "automatic include" limit for all StoreFiles smaller than this value, this value may need to be reduced in write-heavy environments where many files in the 1-2 MB range are being flushed, because every StoreFile will be targeted for compaction and the resulting StoreFiles may still be under the minimum size and require further compaction. If this parameter is lowered, the ratio check is triggered more quickly. This addressed some issues seen in earlier versions of HBase but changing this parameter is no longer necessary in most situations.
-|128 MB
-
-|`hbase.hstore.compaction.max.size`
-| An StoreFile larger than this size will be excluded from compaction. The effect of raising `hbase.hstore.compaction.max.size` is fewer, larger StoreFiles that do not get compacted often. If you feel that compaction is happening too often without much benefit, you can try raising this value.
-|`Long.MAX_VALUE`
-
-|`hbase.hstore.compaction.ratio`
-| For minor compaction, this ratio is used to determine whether a given StoreFile which is larger than `hbase.hstore.compaction.min.size` is eligible for compaction. Its effect is to limit compaction of large StoreFile. The value of `hbase.hstore.compaction.ratio` is expressed as a floating-point decimal.
-
-* A large ratio, such as 10, will produce a single giant StoreFile. Conversely, a value of .25, will produce behavior similar to the BigTable compaction algorithm, producing four StoreFiles.
-* A moderate value of between 1.0 and 1.4 is recommended. When tuning this value, you are balancing write costs with read costs. Raising the value (to something like 1.4) will have more write costs, because you will compact larger StoreFiles. However, during reads, HBase will need to seek through fewer StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of <<bloom>>.
-* Alternatively, you can lower this value to something like 1.0 to reduce the background cost of writes, and use  to limit the number of StoreFiles touched during reads. For most cases, the default value is appropriate.
-| `1.2F`
-
-|`hbase.hstore.compaction.ratio.offpeak`
-| The compaction ratio used during off-peak compactions, if off-peak hours are also configured (see below). Expressed as a floating-point decimal. This allows for more aggressive (or less aggressive, if you set it lower than `hbase.hstore.compaction.ratio`) compaction during a set time period. Ignored if off-peak is disabled (default). This works the same as hbase.hstore.compaction.ratio.
-| `5.0F`
+`hbase.hstore.compaction.min`::
+  The minimum number of StoreFiles which must be eligible for compaction before compaction can run.
+  The goal of tuning `hbase.hstore.compaction.min` is to avoid ending up with too many tiny StoreFiles
+  to compact. Setting this value to 2 would cause a minor compaction each time you have two StoreFiles
+  in a Store, and this is probably not appropriate. If you set this value too high, all the other
+  values will need to be adjusted accordingly. For most cases, the default value is appropriate.
+  In previous versions of HBase, the parameter `hbase.hstore.compaction.min` was called
+  `hbase.hstore.compactionThreshold`.
++
+*Default*: 3
+
+`hbase.hstore.compaction.max`::
+  The maximum number of StoreFiles which will be selected for a single minor compaction,
+  regardless of the number of eligible StoreFiles. Effectively, the value of
+  `hbase.hstore.compaction.max` controls the length of time it takes a single
+  compaction to complete. Setting it larger means that more StoreFiles are included
+  in a compaction. For most cases, the default value is appropriate.
++
+*Default*: 10
+
+`hbase.hstore.compaction.min.size`::
+  A StoreFile smaller than this size will always be eligible for minor compaction.
+  StoreFiles this size or larger are evaluated by `hbase.hstore.compaction.ratio`
+  to determine if they are eligible. Because this limit represents the "automatic
+  include" limit for all StoreFiles smaller than this value, this value may need
+  to be reduced in write-heavy environments where many files in the 1-2 MB range
+  are being flushed, because every StoreFile will be targeted for compaction and
+  the resulting StoreFiles may still be under the minimum size and require further
+  compaction. If this parameter is lowered, the ratio check is triggered more quickly.
+  This addressed some issues seen in earlier versions of HBase but changing this
+  parameter is no longer necessary in most situations.
++
+*Default*:128 MB
 
-| `hbase.offpeak.start.hour`
-| The start of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set to -1 to disable off-peak.
-| `-1` (disabled)
+`hbase.hstore.compaction.max.size`::
+  A StoreFile larger than this size will be excluded from compaction. The effect of
+  raising `hbase.hstore.compaction.max.size` is fewer, larger StoreFiles that do not
+  get compacted often. If you feel that compaction is happening too often without
+  much benefit, you can try raising this value.
++
+*Default*: `Long.MAX_VALUE`
 
-| `hbase.offpeak.end.hour`
-| The end of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set to -1 to disable off-peak.
-| `-1` (disabled)
+`hbase.hstore.compaction.ratio`::
+  For minor compaction, this ratio is used to determine whether a given StoreFile
+  which is larger than `hbase.hstore.compaction.min.size` is eligible for compaction.
+  Its effect is to limit compaction of large StoreFile. The value of
+  `hbase.hstore.compaction.ratio` is expressed as a floating-point decimal.
++
+* A large ratio, such as 10, will produce a single giant StoreFile. Conversely,
+  a value of .25, will produce behavior similar to the BigTable compaction algorithm,
+  producing four StoreFiles.
+* A moderate value of between 1.0 and 1.4 is recommended. When tuning this value,
+  you are balancing write costs with read costs. Raising the value (to something like
+  1.4) will have more write costs, because you will compact larger StoreFiles.
+  However, during reads, HBase will need to seek through fewer StoreFiles to
+  accomplish the read. Consider this approach if you cannot take advantage of <<blooms>>.
+* Alternatively, you can lower this value to something like 1.0 to reduce the
+  background cost of writes, and use  to limit the number of StoreFiles touched
+  during reads. For most cases, the default value is appropriate.
++
+*Default*: `1.2F`
+
+`hbase.hstore.compaction.ratio.offpeak`::
+  The compaction ratio used during off-peak compactions, if off-peak hours are
+  also configured (see below). Expressed as a floating-point decimal. This allows
+  for more aggressive (or less aggressive, if you set it lower than
+  `hbase.hstore.compaction.ratio`) compaction during a set time period. Ignored
+  if off-peak is disabled (default). This works the same as
+  `hbase.hstore.compaction.ratio`.
++
+*Default*: `5.0F`
 
-| `hbase.regionserver.thread.compaction.throttle`
-| There are two different thread pools for compactions, one for large compactions and the other for small compactions. This helps to keep compaction of lean tables (such as `hbase:meta`) fast. If a compaction is larger than this threshold, it goes into the large compaction pool. In most cases, the default value is appropriate.
-| `2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size` (which defaults to `128`)
+`hbase.offpeak.start.hour`::
+  The start of off-peak hours, expressed as an integer between 0 and 23, inclusive.
+  Set to -1 to disable off-peak.
++
+*Default*: `-1` (disabled)
 
-| `hbase.hregion.majorcompaction`
-| Time between major compactions, expressed in milliseconds. Set to 0 to disable time-based automatic major compactions. User-requested and size-based major compactions will still run. This value is multiplied by `hbase.hregion.majorcompaction.jitter` to cause compaction to start at a somewhat-random time during a given window of time.
-| 7 days (`604800000` milliseconds)
+`hbase.offpeak.end.hour`::
+  The end of off-peak hours, expressed as an integer between 0 and 23, inclusive.
+  Set to -1 to disable off-peak.
++
+*Default*: `-1` (disabled)
+
+`hbase.regionserver.thread.compaction.throttle`::
+  There are two different thread pools for compactions, one for large compactions
+  and the other for small compactions. This helps to keep compaction of lean tables
+  (such as `hbase:meta`) fast. If a compaction is larger than this threshold,
+  it goes into the large compaction pool. In most cases, the default value is
+  appropriate.
++
+*Default*: `2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size`
+(which defaults to `128`)
+
+`hbase.hregion.majorcompaction`::
+  Time between major compactions, expressed in milliseconds. Set to 0 to disable
+  time-based automatic major compactions. User-requested and size-based major
+  compactions will still run. This value is multiplied by
+  `hbase.hregion.majorcompaction.jitter` to cause compaction to start at a
+  somewhat-random time during a given window of time.
++
+*Default*: 7 days (`604800000` milliseconds)
 
-| `hbase.hregion.majorcompaction.jitter`
-| A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occur a given amount of time either side of `hbase.hregion.majorcompaction`. The smaller the number, the closer the compactions will happen to the `hbase.hregion.majorcompaction` interval. Expressed as a floating-point decimal.
-| `.50F`
-|===
+`hbase.hregion.majorcompaction.jitter`::
+  A multiplier applied to hbase.hregion.majorcompaction to cause compaction to
+  occur a given amount of time either side of `hbase.hregion.majorcompaction`.
+  The smaller the number, the closer the compactions will happen to the
+  `hbase.hregion.majorcompaction` interval. Expressed as a floating-point decimal.
++
+*Default*: `.50F`
 
 [[compaction.file.selection.old]]
 ===== Compaction File Selection
@@ -1906,8 +2002,8 @@ Why?
 * 100 -> No, because sum(50, 23, 12, 12) * 1.0 = 97.
 * 50 -> No, because sum(23, 12, 12) * 1.0 = 47.
 * 23 -> Yes, because sum(12, 12) * 1.0 = 24.
-* 12 -> Yes, because the previous file has been included, and because this does not exceed the the max-file limit of 5
-* 12 -> Yes, because the previous file had been included, and because this does not exceed the the max-file limit of 5.
+* 12 -> Yes, because the previous file has been included, and because this does not exceed the max-file limit of 5
+* 12 -> Yes, because the previous file had been included, and because this does not exceed the max-file limit of 5.
 
 [[compaction.file.selection.example2]]
 ====== Minor Compaction File Selection - Example #2 (Not Enough Files ToCompact)
@@ -1962,8 +2058,109 @@ Why?
 [[compaction.config.impact]]
 .Impact of Key Configuration Options
 
-NOTE: This information is now included in the configuration parameter table in <<compaction.configuration.parameters>>.
+NOTE: This information is now included in the configuration parameter table in <<compaction.parameters>>.
+
+[[ops.date.tiered]]
+===== Date Tiered Compaction
+
+Date tiered compaction is a date-aware store file compaction strategy that is beneficial for time-range scans for time-series data.
+
+[[ops.date.tiered.when]]
+===== When To Use Date Tiered Compactions
+
+Consider using Date Tiered Compaction for reads for limited time ranges, especially scans of recent data
+
+Don't use it for
+
+* random gets without a limited time range
+* frequent deletes and updates
+* Frequent out of order data writes creating long tails, especially writes with future timestamps
+* frequent bulk loads with heavily overlapping time ranges
+
+.Performance Improvements
+Performance testing has shown that the performance of time-range scans improve greatly for limited time ranges, especially scans of recent data.
+
+[[ops.date.tiered.enable]]
+====== Enabling Date Tiered Compaction
+
+You can enable Date Tiered compaction for a table or a column family, by setting its `hbase.hstore.engine.class` to `org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine`.
+
+You also need to set `hbase.hstore.blockingStoreFiles` to a high number, such as 60, if using all default settings, rather than the default value of 12). Use 1.5~2 x projected file count if changing the parameters, Projected file count = windows per tier x tier count + incoming window min + files older than max age
 
+You also need to set `hbase.hstore.compaction.max` to the same value as `hbase.hstore.blockingStoreFiles` to unblock major compaction.
+
+.Procedure: Enable Date Tiered Compaction
+. Run one of following commands in the HBase shell.
+  Replace the table name `orders_table` with the name of your table.
++
+[source,sql]
+----
+alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}
+alter 'orders_table', {NAME => 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}}
+create 'orders_table', 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine', 'hbase.hstore.blockingStoreFiles' => '60', 'hbase.hstore.compaction.min'=>'2', 'hbase.hstore.compaction.max'=>'60'}
+----
+
+. Configure other options if needed.
+  See <<ops.date.tiered.config>> for more information.
+
+.Procedure: Disable Date Tiered Compaction
+. Set the `hbase.hstore.engine.class` option to either nil or `org.apache.hadoop.hbase.regionserver.DefaultStoreEngine`.
+  Either option has the same effect.
+  Make sure you set the other options you changed to the original settings too.
++
+[source,sql]
+----
+alter 'orders_table', CONFIGURATION => {'hbase.hstore.engine.class' => 'org.apache.hadoop.hbase.regionserver.DefaultStoreEngine'\uff0c 'hbase.hstore.blockingStoreFiles' => '12', 'hbase.hstore.compaction.min'=>'6', 'hbase.hstore.compaction.max'=>'12'}}
+----
+
+When you change the store engine either way, a major compaction will likely be performed on most regions.
+This is not necessary on new tables.
+
+[[ops.date.tiered.config]]
+====== Configuring Date Tiered Compaction
+
+Each of the settings for date tiered compaction should be configured at the table or column family level.
+If you use HBase shell, the general command pattern is as follows:
+
+[source,sql]
+----
+alter 'orders_table', CONFIGURATION => {'key' => 'value', ..., 'key' => 'value'}}
+----
+
+[[ops.date.tiered.config.parameters]]
+.Tier Parameters
+
+You can configure your date tiers by changing the settings for the following parameters:
+
+.Date Tier Parameters
+[cols="1,1a", frame="all", options="header"]
+|===
+| Setting
+| Notes
+
+|`hbase.hstore.compaction.date.tiered.max.storefile.age.millis`
+|Files with max-timestamp smaller than this will no longer be compacted.Default at Long.MAX_VALUE.
+
+| `hbase.hstore.compaction.date.tiered.base.window.millis`
+| Base window size in milliseconds. Default at 6 hours.
+
+| `hbase.hstore.compaction.date.tiered.windows.per.tier`
+| Number of windows per tier. Default at 4.
+
+| `hbase.hstore.compaction.date.tiered.incoming.window.min`
+| Minimal number of files to compact in the incoming window. Set it to expected number of files in the window to avoid wasteful compaction. Default at 6.
+
+| `hbase.hstore.compaction.date.tiered.window.policy.class`
+| The policy to select store files within the same time window. It doesn\u2019t apply to the incoming window. Default at exploring compaction. This is to avoid wasteful compaction.
+|===
+
+[[ops.date.tiered.config.compaction.throttler]]
+.Compaction Throttler
+
+With tiered compaction all servers in the cluster will promote windows to higher tier at the same time, so using a compaction throttle is recommended:
+Set `hbase.regionserver.throughput.controller` to `org.apache.hadoop.hbase.regionserver.compactions.PressureAwareCompactionThroughputController`.
+
+NOTE: For more information about date tiered compaction, please refer to the design specification at https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8
 [[ops.stripe]]
 ===== Experimental: Stripe Compactions
 
@@ -2002,7 +2199,6 @@ You can enable stripe compaction for a table or a column family, by setting its
 You also need to set the `hbase.hstore.blockingStoreFiles` to a high number, such as 100 (rather than the default value of 10).
 
 .Procedure: Enable Stripe Compaction
-. If the table already exists, disable the table.
 . Run one of following commands in the HBase shell.
   Replace the table name `orders_table` with the name of your table.
 +
@@ -2018,7 +2214,6 @@ create 'orders_table', 'blobs_cf', CONFIGURATION => {'hbase.hstore.engine.class'
 . Enable the table.
 
 .Procedure: Disable Stripe Compaction
-. Disable the table.
 . Set the `hbase.hstore.engine.class` option to either nil or `org.apache.hadoop.hbase.regionserver.DefaultStoreEngine`.
   Either option has the same effect.
 +
@@ -2035,7 +2230,7 @@ This is not necessary on new tables.
 [[ops.stripe.config]]
 ====== Configuring Stripe Compaction
 
-Each of the settings for stripe compaction should be configured at the table or column family, after disabling the table.
+Each of the settings for stripe compaction should be configured at the table or column family level.
 If you use HBase shell, the general command pattern is as follows:
 
 [source,sql]
@@ -2100,7 +2295,7 @@ When at least `hbase.store.stripe.compaction.minFilesL0` such files (by default,
 [[ops.stripe.config.compact]]
 .Normal Compaction Configuration and Stripe Compaction
 
-All the settings that apply to normal compactions (see <<compaction.configuration.parameters>>) apply to stripe compactions.
+All the settings that apply to normal compactions (see <<compaction.parameters>>) apply to stripe compactions.
 The exceptions are the minimum and maximum number of files, which are set to higher values by default because the files in stripes are smaller.
 To control these for stripe compactions, use `hbase.store.stripe.compaction.minFiles` and `hbase.store.stripe.compaction.maxFiles`, rather than `hbase.hstore.compaction.min` and `hbase.hstore.compaction.max`.
 
@@ -2168,7 +2363,7 @@ See link:http://blog.cloudera.com/blog/2013/09/how-to-use-hbase-bulk-loading-and
 [[arch.bulk.load.adv]]
 === Advanced Usage
 
-Although the `importtsv` tool is useful in many cases, advanced users may want to generate data programatically, or import data from other formats.
+Although the `importtsv` tool is useful in many cases, advanced users may want to generate data programmatically, or import data from other formats.
 To get started doing so, dig into `ImportTsv.java` and check the JavaDoc for HFileOutputFormat.
 
 The import step of the bulk load can also be done programmatically.
@@ -2264,8 +2459,8 @@ In terms of semantics, TIMELINE consistency as implemented by HBase differs from
 .Timeline Consistency
 image::timeline_consistency.png[Timeline Consistency]
 
-To better understand the TIMELINE semantics, lets look at the above diagram.
-Lets say that there are two clients, and the first one writes x=1 at first, then x=2 and x=3 later.
+To better understand the TIMELINE semantics, let's look at the above diagram.
+Let's say that there are two clients, and the first one writes x=1 at first, then x=2 and x=3 later.
 As above, all writes are handled by the primary region replica.
 The writes are saved in the write ahead log (WAL), and replicated to the other replicas asynchronously.
 In the above diagram, notice that replica_id=1 received 2 updates, and its data shows that x=2, while the replica_id=2 only received a single update, and its data shows that x=1.
@@ -2298,18 +2493,18 @@ To serve the region data from multiple replicas, HBase opens the regions in seco
 The regions opened in secondary mode will share the same data files with the primary region replica, however each secondary region replica will have its own MemStore to keep the unflushed data (only primary region can do flushes). Also to serve reads from secondary regions, the blocks of data files may be also cached in the block caches for the secondary regions.
 
 === Where is the code
-This feature is delivered in two phases, Phase 1 and 2. The first phase is done in time for HBase-1.0.0 release. Meaning that using HBase-1.0.x, you can use all the features that are marked for Phase 1. Phase 2 is committed in HBase-1.1.0, meaning all HBase versions after 1.1.0 should contain Phase 2 items. 
+This feature is delivered in two phases, Phase 1 and 2. The first phase is done in time for HBase-1.0.0 release. Meaning that using HBase-1.0.x, you can use all the features that are marked for Phase 1. Phase 2 is committed in HBase-1.1.0, meaning all HBase versions after 1.1.0 should contain Phase 2 items.
 
 === Propagating writes to region replicas
-As discussed above writes only go to the primary region replica. For propagating the writes from the primary region replica to the secondaries, there are two different mechanisms. For read-only tables, you do not need to use any of the following methods. Disabling and enabling the table should make the data available in all region replicas. For mutable tables, you have to use *only* one of the following mechanisms: storefile refresher, or async wal replication. The latter is recommeded. 
+As discussed above writes only go to the primary region replica. For propagating the writes from the primary region replica to the secondaries, there are two different mechanisms. For read-only tables, you do not need to use any of the following methods. Disabling and enabling the table should make the data available in all region replicas. For mutable tables, you have to use *only* one of the following mechanisms: storefile refresher, or async wal replication. The latter is recommended.
 
 ==== StoreFile Refresher
-The first mechanism is store file refresher which is introduced in HBase-1.0+. Store file refresher is a thread per region server, which runs periodically, and does a refresh operation for the store files of the primary region for the secondary region replicas. If enabled, the refresher will ensure that the secondary region replicas see the new flushed, compacted or bulk loaded files from the primary region in a timely manner. However, this means that only flushed data can be read back from the secondary region replicas, and after the refresher is run, making the secondaries lag behind the primary for an a longer time. 
+The first mechanism is store file refresher which is introduced in HBase-1.0+. Store file refresher is a thread per region server, which runs periodically, and does a refresh operation for the store files of the primary region for the secondary region replicas. If enabled, the refresher will ensure that the secondary region replicas see the new flushed, compacted or bulk loaded files from the primary region in a timely manner. However, this means that only flushed data can be read back from the secondary region replicas, and after the refresher is run, making the secondaries lag behind the primary for an a longer time.
 
-For turning this feature on, you should configure `hbase.regionserver.storefile.refresh.period` to a non-zero value. See Configuration section below. 
+For turning this feature on, you should configure `hbase.regionserver.storefile.refresh.period` to a non-zero value. See Configuration section below.
 
 ==== Asnyc WAL replication
-The second mechanism for propagation of writes to secondaries is done via \u201cAsync WAL Replication\u201d feature and is only available in HBase-1.1+. This works similarly to HBase\u2019s multi-datacenter replication, but instead the data from a region is replicated to the secondary regions. Each secondary replica always receives and observes the writes in the same order that the primary region committed them. In some sense, this design can be thought of as \u201cin-cluster replication\u201d, where instead of replicating to a different datacenter, the data goes to secondary regions to keep secondary region\u2019s in-memory state up to date. The data files are shared between the primary region and the other replicas, so that there is no extra storage overhead. However, the secondary regions will have recent non-flushed data in their memstores, which increases the memory overhead. The primary region writes flush, compaction, and bulk load events to its WAL as well, which are also replicated through w
 al replication to secondaries. When they observe the flush/compaction or bulk load event, the secondary regions replay the event to pick up the new files and drop the old ones.  
+The second mechanism for propagation of writes to secondaries is done via \u201cAsync WAL Replication\u201d feature and is only available in HBase-1.1+. This works similarly to HBase\u2019s multi-datacenter replication, but instead the data from a region is replicated to the secondary regions. Each secondary replica always receives and observes the writes in the same order that the primary region committed them. In some sense, this design can be thought of as \u201cin-cluster replication\u201d, where instead of replicating to a different datacenter, the data goes to secondary regions to keep secondary region\u2019s in-memory state up to date. The data files are shared between the primary region and the other replicas, so that there is no extra storage overhead. However, the secondary regions will have recent non-flushed data in their memstores, which increases the memory overhead. The primary region writes flush, compaction, and bulk load events to its WAL as well, which are also replicated through w
 al replication to secondaries. When they observe the flush/compaction or bulk load event, the secondary regions replay the event to pick up the new files and drop the old ones.
 
 Committing writes in the same order as in primary ensures that the secondaries won\u2019t diverge from the primary regions data, but since the log replication is asynchronous, the data might still be stale in secondary regions. Since this feature works as a replication endpoint, the performance and latency characteristics is expected to be similar to inter-cluster replication.
 
@@ -2322,18 +2517,18 @@ Asyn WAL Replication feature will add a new replication peer named `region_repli
 	hbase> disable_peer 'region_replica_replication'
 ----
 
-=== Store File TTL 
-In both of the write propagation approaches mentioned above, store files of the primary will be opened in secondaries independent of the primary region. So for files that the primary compacted away, the secondaries might still be referring to these files for reading. Both features are using HFileLinks to refer to files, but there is no protection (yet) for guaranteeing that the file will not be deleted prematurely. Thus, as a guard, you should set the configuration property `hbase.master.hfilecleaner.ttl` to a larger value, such as 1 hour to guarantee that you will not receive IOExceptions for requests going to replicas. 
+=== Store File TTL
+In both of the write propagation approaches mentioned above, store files of the primary will be opened in secondaries independent of the primary region. So for files that the primary compacted away, the secondaries might still be referring to these files for reading. Both features are using HFileLinks to refer to files, but there is no protection (yet) for guaranteeing that the file will not be deleted prematurely. Thus, as a guard, you should set the configuration property `hbase.master.hfilecleaner.ttl` to a larger value, such as 1 hour to guarantee that you will not receive IOExceptions for requests going to replicas.
 
 === Region replication for META table\u2019s region
-Currently, Async WAL Replication is not done for the META table\u2019s WAL. The meta table\u2019s secondary replicas still refreshes themselves from the persistent store files. Hence the `hbase.regionserver.meta.storefile.refresh.period` needs to be set to a certain non-zero value for refreshing the meta store files. Note that this configuration is configured differently than 
-`hbase.regionserver.storefile.refresh.period`. 
+Currently, Async WAL Replication is not done for the META table\u2019s WAL. The meta table\u2019s secondary replicas still refreshes themselves from the persistent store files. Hence the `hbase.regionserver.meta.storefile.refresh.period` needs to be set to a certain non-zero value for refreshing the meta store files. Note that this configuration is configured differently than
+`hbase.regionserver.storefile.refresh.period`.
 
 === Memory accounting
 The secondary region replicas refer to the data files of the primary region replica, but they have their own memstores (in HBase-1.1+) and uses block cache as well. However, one distinction is that the secondary region replicas cannot flush the data when there is memory pressure for their memstores. They can only free up memstore memory when the primary region does a flush and this flush is replicated to the secondary. Since in a region server hosting primary replicas for some regions and secondaries for some others, the secondaries might cause extra flushes to the primary regions in the same host. In extreme situations, there can be no memory left for adding new writes coming from the primary via wal replication. For unblocking this situation (and since secondary cannot flush by itself), the secondary is allowed to do a \u201cstore file refresh\u201d by doing a file system list operation to pick up new files from primary, and possibly dropping its memstore. This refresh will only be perf
 ormed if the memstore size of the biggest secondary region replica is at least `hbase.region.replica.storefile.refresh.memstore.multiplier` (default 4) times bigger than the biggest memstore of a primary replica. One caveat is that if this is performed, the secondary can observe partial row updates across column families (since column families are flushed independently). The default should be good to not do this operation frequently. You can set this value to a large number to disable this feature if desired, but be warned that it might cause the replication to block forever.
 
 === Secondary replica failover
-When a secondary region replica first comes online, or fails over, it may have served some edits from it\u2019s memstore. Since the recovery is handled differently for secondary replicas, the secondary has to ensure that it does not go back in time before it starts serving requests after assignment. For doing that, the secondary waits until it observes a full flush cycle (start flush, commit flush) or a \u201cregion open event\u201d replicated from the primary. Until this happens, the secondary region replica will reject all read requests by throwing an IOException with message \u201cThe region's reads are disabled\u201d. However, the other replicas will probably still be available to read, thus not causing any impact for the rpc with TIMELINE consistency. To facilitate faster recovery, the secondary region will trigger a flush request from the primary when it is opened. The configuration property `hbase.region.replica.wait.for.primary.flush` (enabled by default) can be used to disable this featur
 e if needed. 
+When a secondary region replica first comes online, or fails over, it may have served some edits from its memstore. Since the recovery is handled differently for secondary replicas, the secondary has to ensure that it does not go back in time before it starts serving requests after assignment. For doing that, the secondary waits until it observes a full flush cycle (start flush, commit flush) or a \u201cregion open event\u201d replicated from the primary. Until this happens, the secondary region replica will reject all read requests by throwing an IOException with message \u201cThe region's reads are disabled\u201d. However, the other replicas will probably still be available to read, thus not causing any impact for the rpc with TIMELINE consistency. To facilitate faster recovery, the secondary region will trigger a flush request from the primary when it is opened. The configuration property `hbase.region.replica.wait.for.primary.flush` (enabled by default) can be used to disable this feature i
 f needed.
 
 
 
@@ -2342,7 +2537,7 @@ When a secondary region replica first comes online, or fails over, it may have s
 
 To use highly available reads, you should set the following properties in `hbase-site.xml` file.
 There is no specific configuration to enable or disable region replicas.
-Instead you can change the number of region replicas per table to increase or decrease at the table creation or with alter table. The following configuration is for using async wal replication and using meta replicas of 3. 
+Instead you can change the number of region replicas per table to increase or decrease at the table creation or with alter table. The following configuration is for using async wal replication and using meta replicas of 3.
 
 
 ==== Server side properties
@@ -2369,7 +2564,7 @@ Instead you can change the number of region replicas per table to increase or de
     <name>hbase.region.replica.replication.enabled</name>
     <value>true</value>
     <description>
-      Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named "region_replica_replication" will be created which will tail the logs and replicate the mutatations to region replicas for tables that have region replication > 1. If this is enabled once, disabling this replication also      requires disabling the replication peer using shell or ReplicationAdmin java class. Replication to secondary region replicas works over standard inter-cluster replication. So replication, if disabled explicitly, also has to be enabled by setting "hbase.replication"� to true for this feature to work.
+      Whether asynchronous WAL replication to the secondary region replicas is enabled or not. If this is enabled, a replication peer named "region_replica_replication" will be created which will tail the logs and replicate the mutations to region replicas for tables that have region replication > 1. If this is enabled once, disabling this replication also      requires disabling the replication peer using shell or ReplicationAdmin java class. Replication to secondary region replicas works over standard inter-cluster replication.
     </description>
 </property>
 <property>
@@ -2403,7 +2598,7 @@ Instead you can change the number of region replicas per table to increase or de
 </property>
 
 
-<property> 
+<property>
     <name>hbase.region.replica.storefile.refresh.memstore.multiplier</name>
     <value>4</value>
     <description>
@@ -2466,7 +2661,7 @@ Ensure to set the following for all clients (and servers) that will use region r
 </property>
 ----
 
-Note HBase-1.0.x users should use `hbase.ipc.client.allowsInterrupt` rather than `hbase.ipc.client.specificThreadForWriting`. 
+Note HBase-1.0.x users should use `hbase.ipc.client.allowsInterrupt` rather than `hbase.ipc.client.specificThreadForWriting`.
 
 === User Interface
 
@@ -2537,7 +2732,7 @@ hbase> scan 't1', {CONSISTENCY => 'TIMELINE'}
 
 ==== Java
 
-You can set set the consistency for Gets and Scans and do requests as follows.
+You can set the consistency for Gets and Scans and do requests as follows.
 
 [source,java]
 ----

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/asf.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/asf.adoc b/src/main/asciidoc/_chapters/asf.adoc
index 77eed8f..47c29e5 100644
--- a/src/main/asciidoc/_chapters/asf.adoc
+++ b/src/main/asciidoc/_chapters/asf.adoc
@@ -35,13 +35,13 @@ HBase is a project in the Apache Software Foundation and as such there are respo
 [[asf.devprocess]]
 === ASF Development Process
 
-See the link:http://www.apache.org/dev/#committers[Apache Development Process page]            for all sorts of information on how the ASF is structured (e.g., PMC, committers, contributors), to tips on contributing and getting involved, and how open-source works at ASF. 
+See the link:http://www.apache.org/dev/#committers[Apache Development Process page]            for all sorts of information on how the ASF is structured (e.g., PMC, committers, contributors), to tips on contributing and getting involved, and how open-source works at ASF.
 
 [[asf.reporting]]
 === ASF Board Reporting
 
 Once a quarter, each project in the ASF portfolio submits a report to the ASF board.
 This is done by the HBase project lead and the committers.
-See link:http://www.apache.org/foundation/board/reporting[ASF board reporting] for more information. 
+See link:http://www.apache.org/foundation/board/reporting[ASF board reporting] for more information.
 
 :numbered:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/case_studies.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/case_studies.adoc b/src/main/asciidoc/_chapters/case_studies.adoc
index 992414c..b021aa2 100644
--- a/src/main/asciidoc/_chapters/case_studies.adoc
+++ b/src/main/asciidoc/_chapters/case_studies.adoc
@@ -55,7 +55,7 @@ These jobs were consistently found to be waiting on map and reduce tasks assigne
 
 .Datanodes:
 * Two 12-core processors
-* Six Enerprise SATA disks
+* Six Enterprise SATA disks
 * 24GB of RAM
 * Two bonded gigabit NICs
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/community.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/community.adoc b/src/main/asciidoc/_chapters/community.adoc
index 4b91b0d..ba07df7 100644
--- a/src/main/asciidoc/_chapters/community.adoc
+++ b/src/main/asciidoc/_chapters/community.adoc
@@ -45,35 +45,35 @@ See link:http://search-hadoop.com/m/asM982C5FkS1[HBase, mail # dev - Thoughts
 
 The below policy is something we put in place 09/2012.
 It is a suggested policy rather than a hard requirement.
-We want to try it first to see if it works before we cast it in stone. 
+We want to try it first to see if it works before we cast it in stone.
 
 Apache HBase is made of link:https://issues.apache.org/jira/browse/HBASE#selectedTab=com.atlassian.jira.plugin.system.project%3Acomponents-panel[components].
 Components have one or more <<owner,OWNER>>s.
-See the 'Description' field on the link:https://issues.apache.org/jira/browse/HBASE#selectedTab=com.atlassian.jira.plugin.system.project%3Acomponents-panel[components]        JIRA page for who the current owners are by component. 
+See the 'Description' field on the link:https://issues.apache.org/jira/browse/HBASE#selectedTab=com.atlassian.jira.plugin.system.project%3Acomponents-panel[components]        JIRA page for who the current owners are by component.
 
 Patches that fit within the scope of a single Apache HBase component require, at least, a +1 by one of the component's owners before commit.
-If owners are absent -- busy or otherwise -- two +1s by non-owners will suffice. 
+If owners are absent -- busy or otherwise -- two +1s by non-owners will suffice.
 
-Patches that span components need at least two +1s before they can be committed, preferably +1s by owners of components touched by the x-component patch (TODO: This needs tightening up but I think fine for first pass). 
+Patches that span components need at least two +1s before they can be committed, preferably +1s by owners of components touched by the x-component patch (TODO: This needs tightening up but I think fine for first pass).
 
-Any -1 on a patch by anyone vetos a patch; it cannot be committed until the justification for the -1 is addressed. 
+Any -1 on a patch by anyone vetoes a patch; it cannot be committed until the justification for the -1 is addressed.
 
 [[hbase.fix.version.in.jira]]
 .How to set fix version in JIRA on issue resolve
 
 Here is how link:http://search-hadoop.com/m/azemIi5RCJ1[we agreed] to set versions in JIRA when we resolve an issue.
-If trunk is going to be 0.98.0 then: 
+If master is going to be 0.98.0 then:
 
-* Commit only to trunk: Mark with 0.98 
-* Commit to 0.95 and trunk : Mark with 0.98, and 0.95.x 
-* Commit to 0.94.x and 0.95, and trunk: Mark with 0.98, 0.95.x, and 0.94.x 
-* Commit to 89-fb: Mark with 89-fb. 
-* Commit site fixes: no version 
+* Commit only to master: Mark with 0.98
+* Commit to 0.95 and master: Mark with 0.98, and 0.95.x
+* Commit to 0.94.x and 0.95, and master: Mark with 0.98, 0.95.x, and 0.94.x
+* Commit to 89-fb: Mark with 89-fb.
+* Commit site fixes: no version
 
 [[hbase.when.to.close.jira]]
 .Policy on when to set a RESOLVED JIRA as CLOSED
 
-We link:http://search-hadoop.com/m/4cIKs1iwXMS1[agreed] that for issues that list multiple releases in their _Fix Version/s_ field, CLOSE the issue on the release of any of the versions listed; subsequent change to the issue must happen in a new JIRA. 
+We link:http://search-hadoop.com/m/4cIKs1iwXMS1[agreed] that for issues that list multiple releases in their _Fix Version/s_ field, CLOSE the issue on the release of any of the versions listed; subsequent change to the issue must happen in a new JIRA.
 
 [[no.permanent.state.in.zk]]
 .Only transient state in ZooKeeper!
@@ -81,7 +81,7 @@ We link:http://search-hadoop.com/m/4cIKs1iwXMS1[agreed] that for issues that lis
 You should be able to kill the data in zookeeper and hbase should ride over it recreating the zk content as it goes.
 This is an old adage around these parts.
 We just made note of it now.
-We also are currently in violation of this basic tenet -- replication at least keeps permanent state in zk -- but we are working to undo this breaking of a golden rule. 
+We also are currently in violation of this basic tenet -- replication at least keeps permanent state in zk -- but we are working to undo this breaking of a golden rule.
 
 [[community.roles]]
 == Community Roles
@@ -90,22 +90,22 @@ We also are currently in violation of this basic tenet -- replication at least k
 .Component Owner/Lieutenant
 
 Component owners are listed in the description field on this Apache HBase JIRA link:https://issues.apache.org/jira/browse/HBASE#selectedTab=com.atlassian.jira.plugin.system.project%3Acomponents-panel[components]        page.
-The owners are listed in the 'Description' field rather than in the 'Component Lead' field because the latter only allows us list one individual whereas it is encouraged that components have multiple owners. 
+The owners are listed in the 'Description' field rather than in the 'Component Lead' field because the latter only allows us list one individual whereas it is encouraged that components have multiple owners.
 
-Owners or component lieutenants are volunteers who are (usually, but not necessarily) expert in their component domain and may have an agenda on how they think their Apache HBase component should evolve. 
+Owners or component lieutenants are volunteers who are (usually, but not necessarily) expert in their component domain and may have an agenda on how they think their Apache HBase component should evolve.
 
-. Owners will try and review patches that land within their component's scope. 
-. If applicable, if an owner has an agenda, they will publish their goals or the design toward which they are driving their component 
+. Owners will try and review patches that land within their component's scope.
+. If applicable, if an owner has an agenda, they will publish their goals or the design toward which they are driving their component
 
 If you would like to be volunteer as a component owner, just write the dev list and we'll sign you up.
-Owners do not need to be committers. 
+Owners do not need to be committers.
 
 [[hbase.commit.msg.format]]
 == Commit Message format
 
-We link:http://search-hadoop.com/m/Gwxwl10cFHa1[agreed] to the following SVN commit message format: 
+We link:http://search-hadoop.com/m/Gwxwl10cFHa1[agreed] to the following Git commit message format:
 [source]
 ----
 HBASE-xxxxx <title>. (<contributor>)
----- 
-If the person making the commit is the contributor, leave off the '(<contributor>)' element. 
+----
+If the person making the commit is the contributor, leave off the '(<contributor>)' element.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/compression.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/compression.adoc b/src/main/asciidoc/_chapters/compression.adoc
index 42d4de5..e5b9b8f 100644
--- a/src/main/asciidoc/_chapters/compression.adoc
+++ b/src/main/asciidoc/_chapters/compression.adoc
@@ -122,6 +122,7 @@ For more details about Prefix Tree encoding, see link:https://issues.apache.org/
 +
 It is difficult to graphically illustrate a prefix tree, so no image is included. See the Wikipedia article for link:http://en.wikipedia.org/wiki/Trie[Trie] for more general information about this data structure.
 
+[[data.block.encoding.types]]
 === Which Compressor or Data Block Encoder To Use
 
 The compression or codec type to use depends on the characteristics of your data. Choosing the wrong type could cause your data to take more space rather than less, and can have performance implications.
@@ -142,17 +143,26 @@ In general, you need to weigh your options between smaller size and faster compr
 [[hadoop.native.lib]]
 === Making use of Hadoop Native Libraries in HBase
 
-The Hadoop shared library has a bunch of facility including compression libraries and fast crc'ing. To make this facility available to HBase, do the following. HBase/Hadoop will fall back to use alternatives if it cannot find the native library versions -- or fail outright if you asking for an explicit compressor and there is no alternative available.
+The Hadoop shared library has a bunch of facility including compression libraries and fast crc'ing -- hardware crc'ing if your chipset supports it.
+To make this facility available to HBase, do the following. HBase/Hadoop will fall back to use alternatives if it cannot find the native library
+versions -- or fail outright if you asking for an explicit compressor and there is no alternative available.
 
-If you see the following in your HBase logs, you know that HBase was unable to locate the Hadoop native libraries: 
+First make sure of your Hadoop. Fix this message if you are seeing it starting Hadoop processes:
+----
+16/02/09 22:40:24 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
+----
+It means is not properly pointing at its native libraries or the native libs were compiled for another platform.
+Fix this first.
+
+Then if you see the following in your HBase logs, you know that HBase was unable to locate the Hadoop native libraries:
 [source]
 ----
 2014-08-07 09:26:20,139 WARN  [main] util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
-----      
-If the libraries loaded successfully, the WARN message does not show. 
+----
+If the libraries loaded successfully, the WARN message does not show. Usually this means you are good to go but read on.
 
-Lets presume your Hadoop shipped with a native library that suits the platform you are running HBase on.
-To check if the Hadoop native library is available to HBase, run the following tool (available in  Hadoop 2.1 and greater): 
+Let's presume your Hadoop shipped with a native library that suits the platform you are running HBase on.
+To check if the Hadoop native library is available to HBase, run the following tool (available in  Hadoop 2.1 and greater):
 [source]
 ----
 $ ./bin/hbase --config ~/conf_hbase org.apache.hadoop.util.NativeLibraryChecker
@@ -165,28 +175,54 @@ lz4:    false
 bzip2:  false
 2014-08-26 13:15:38,863 INFO  [main] util.ExitUtil: Exiting with status 1
 ----
-Above shows that the native hadoop library is not available in HBase context. 
+Above shows that the native hadoop library is not available in HBase context.
+
+The above NativeLibraryChecker tool may come back saying all is hunky-dory
+-- i.e. all libs show 'true', that they are available -- but follow the below
+presecription anyways to ensure the native libs are available in HBase context,
+when it goes to use them.
 
 To fix the above, either copy the Hadoop native libraries local or symlink to them if the Hadoop and HBase stalls are adjacent in the filesystem.
-You could also point at their location by setting the `LD_LIBRARY_PATH` environment variable.
+You could also point at their location by setting the `LD_LIBRARY_PATH` environment variable in your hbase-env.sh.
 
-Where the JVM looks to find native librarys is "system dependent" (See `java.lang.System#loadLibrary(name)`). On linux, by default, is going to look in _lib/native/PLATFORM_ where `PLATFORM`      is the label for the platform your HBase is installed on.
+Where the JVM looks to find native libraries is "system dependent" (See `java.lang.System#loadLibrary(name)`). On linux, by default, is going to look in _lib/native/PLATFORM_ where `PLATFORM`      is the label for the platform your HBase is installed on.
 On a local linux machine, it seems to be the concatenation of the java properties `os.name` and `os.arch` followed by whether 32 or 64 bit.
 HBase on startup prints out all of the java system properties so find the os.name and os.arch in the log.
-For example: 
+For example:
 [source]
 ----
 ...
 2014-08-06 15:27:22,853 INFO  [main] zookeeper.ZooKeeper: Client environment:os.name=Linux
 2014-08-06 15:27:22,853 INFO  [main] zookeeper.ZooKeeper: Client environment:os.arch=amd64
 ...
-----     
+----
 So in this case, the PLATFORM string is `Linux-amd64-64`.
 Copying the Hadoop native libraries or symlinking at _lib/native/Linux-amd64-64_     will ensure they are found.
-Check with the Hadoop _NativeLibraryChecker_.
- 
+Rolling restart after you have made this change.
+
+Here is an example of how you would set up the symlinks.
+Let the hadoop and hbase installs be in your home directory. Assume your hadoop native libs
+are at ~/hadoop/lib/native. Assume you are on a Linux-amd64-64 platform. In this case,
+you would do the following to link the hadoop native lib so hbase could find them.
+----
+...
+$ mkdir -p ~/hbaseLinux-amd64-64 -> /home/stack/hadoop/lib/native/lib/native/
+$ cd ~/hbase/lib/native/
+$ ln -s ~/hadoop/lib/native Linux-amd64-64
+$ ls -la
+# Linux-amd64-64 -> /home/USER/hadoop/lib/native
+...
+----
+
+If you see PureJavaCrc32C in a stack track or if you see something like the below in a perf trace, then native is not working; you are using the java CRC functions rather than native:
+----
+  5.02%  perf-53601.map      [.] Lorg/apache/hadoop/util/PureJavaCrc32C;.update
+----
+See link:https://issues.apache.org/jira/browse/HBASE-11927[HBASE-11927 Use Native Hadoop Library for HFile checksum (And flip default from CRC32 to CRC32C)],
+for more on native checksumming support. See in particular the release note for how to check if your hardware to see if your processor has support for hardware CRCs.
+Or checkout the Apache link:https://blogs.apache.org/hbase/entry/saving_cpu_using_native_hadoop[Checksums in HBase] blog post.
 
-Here is example of how to point at the Hadoop libs with `LD_LIBRARY_PATH`      environment variable: 
+Here is example of how to point at the Hadoop libs with `LD_LIBRARY_PATH`      environment variable:
 [source]
 ----
 $ LD_LIBRARY_PATH=~/hadoop-2.5.0-SNAPSHOT/lib/native ./bin/hbase --config ~/conf_hbase org.apache.hadoop.util.NativeLibraryChecker
@@ -199,7 +235,7 @@ snappy: true /usr/lib64/libsnappy.so.1
 lz4:    true revision:99
 bzip2:  true /lib64/libbz2.so.1
 ----
-Set in _hbase-env.sh_ the LD_LIBRARY_PATH environment variable when starting your HBase. 
+Set in _hbase-env.sh_ the LD_LIBRARY_PATH environment variable when starting your HBase.
 
 === Compressor Configuration, Installation, and Use
 
@@ -210,13 +246,13 @@ Before HBase can use a given compressor, its libraries need to be available.
 Due to licensing issues, only GZ compression is available to HBase (via native Java libraries) in a default installation.
 Other compression libraries are available via the shared library bundled with your hadoop.
 The hadoop native library needs to be findable when HBase starts.
-See 
+See
 
 .Compressor Support On the Master
 
 A new configuration setting was introduced in HBase 0.95, to check the Master to determine which data block encoders are installed and configured on it, and assume that the entire cluster is configured the same.
 This option, `hbase.master.check.compression`, defaults to `true`.
-This prevents the situation described in link:https://issues.apache.org/jira/browse/HBASE-6370[HBASE-6370], where a table is created or modified to support a codec that a region server does not support, leading to failures that take a long time to occur and are difficult to debug. 
+This prevents the situation described in link:https://issues.apache.org/jira/browse/HBASE-6370[HBASE-6370], where a table is created or modified to support a codec that a region server does not support, leading to failures that take a long time to occur and are difficult to debug.
 
 If `hbase.master.check.compression` is enabled, libraries for all desired compressors need to be installed and configured on the Master, even if the Master does not run a region server.
 
@@ -232,7 +268,7 @@ See <<brand.new.compressor,brand.new.compressor>>).
 
 HBase cannot ship with LZO because of incompatibility between HBase, which uses an Apache Software License (ASL) and LZO, which uses a GPL license.
 See the link:http://wiki.apache.org/hadoop/UsingLzoCompression[Using LZO
-              Compression] wiki page for information on configuring LZO support for HBase. 
+              Compression] wiki page for information on configuring LZO support for HBase.
 
 If you depend upon LZO compression, consider configuring your RegionServers to fail to start if LZO is not available.
 See <<hbase.regionserver.codecs,hbase.regionserver.codecs>>.
@@ -242,21 +278,21 @@ See <<hbase.regionserver.codecs,hbase.regionserver.codecs>>.
 
 LZ4 support is bundled with Hadoop.
 Make sure the hadoop shared library (libhadoop.so) is accessible when you start HBase.
-After configuring your platform (see <<hbase.native.platform,hbase.native.platform>>), you can make a symbolic link from HBase to the native Hadoop libraries.
+After configuring your platform (see <<hadoop.native.lib,hadoop.native.lib>>), you can make a symbolic link from HBase to the native Hadoop libraries.
 This assumes the two software installs are colocated.
-For example, if my 'platform' is Linux-amd64-64: 
+For example, if my 'platform' is Linux-amd64-64:
 [source,bourne]
 ----
 $ cd $HBASE_HOME
 $ mkdir lib/native
 $ ln -s $HADOOP_HOME/lib/native lib/native/Linux-amd64-64
-----            
+----
 Use the compression tool to check that LZ4 is installed on all nodes.
 Start up (or restart) HBase.
-Afterward, you can create and alter tables to enable LZ4 as a compression codec.: 
+Afterward, you can create and alter tables to enable LZ4 as a compression codec.:
 ----
 hbase(main):003:0> alter 'TestTable', {NAME => 'info', COMPRESSION => 'LZ4'}
-----          
+----
 
 [[snappy.compression.installation]]
 .Install Snappy Support
@@ -347,7 +383,7 @@ You must specify either `-write` or `-update-read` as your first parameter, and
 ====
 ----
 
-$ bin/hbase org.apache.hadoop.hbase.util.LoadTestTool -h            
+$ bin/hbase org.apache.hadoop.hbase.util.LoadTestTool -h
 usage: bin/hbase org.apache.hadoop.hbase.util.LoadTestTool <options>
 Options:
  -batchupdate                 Whether to use batch as opposed to separate


[4/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/external_apis.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/external_apis.adoc b/src/main/asciidoc/_chapters/external_apis.adoc
index 37156ca..556c4e0 100644
--- a/src/main/asciidoc/_chapters/external_apis.adoc
+++ b/src/main/asciidoc/_chapters/external_apis.adoc
@@ -27,32 +27,592 @@
 :icons: font
 :experimental:
 
-This chapter will cover access to Apache HBase either through non-Java languages, or through custom protocols.
-For information on using the native HBase APIs, refer to link:http://hbase.apache.org/apidocs/index.html[User API Reference] and the new <<hbase_apis,HBase APIs>> chapter.
+This chapter will cover access to Apache HBase either through non-Java languages and
+through custom protocols. For information on using the native HBase APIs, refer to
+link:http://hbase.apache.org/apidocs/index.html[User API Reference] and the
+<<hbase_apis,HBase APIs>> chapter.
 
-[[nonjava.jvm]]
-== Non-Java Languages Talking to the JVM
+== REST
 
-Currently the documentation on this topic is in the link:http://wiki.apache.org/hadoop/Hbase[Apache HBase Wiki].
-See also the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/thrift/package-summary.html#package_description[Thrift API Javadoc].
+Representational State Transfer (REST) was introduced in 2000 in the doctoral
+dissertation of Roy Fielding, one of the principal authors of the HTTP specification.
 
-== REST
+REST itself is out of the scope of this documentation, but in general, REST allows
+client-server interactions via an API that is tied to the URL itself. This section
+discusses how to configure and run the REST server included with HBase, which exposes
+HBase tables, rows, cells, and metadata as URL specified resources.
+There is also a nice series of blogs on
+link:http://blog.cloudera.com/blog/2013/03/how-to-use-the-apache-hbase-rest-interface-part-1/[How-to: Use the Apache HBase REST Interface]
+by Jesse Anderson.
 
-Currently most of the documentation on REST exists in the link:http://wiki.apache.org/hadoop/Hbase/Stargate[Apache HBase Wiki on REST] (The REST gateway used to be called 'Stargate').  There are also a nice set of blogs on link:http://blog.cloudera.com/blog/2013/03/how-to-use-the-apache-hbase-rest-interface-part-1/[How-to: Use the Apache HBase REST Interface] by Jesse Anderson.
+=== Starting and Stopping the REST Server
 
-To run your REST server under SSL, set `hbase.rest.ssl.enabled` to `true` and also set the following configs when you launch the REST server: (See example commands in <<jmx_config,JMX config>>)
+The included REST server can run as a daemon which starts an embedded Jetty
+servlet container and deploys the servlet into it. Use one of the following commands
+to start the REST server in the foreground or background. The port is optional, and
+defaults to 8080.
 
-[source]
+[source, bash]
 ----
-hbase.rest.ssl.keystore.store
-hbase.rest.ssl.keystore.password
-hbase.rest.ssl.keystore.keypassword
+# Foreground
+$ bin/hbase rest start -p <port>
+
+# Background, logging to a file in $HBASE_LOGS_DIR
+$ bin/hbase-daemon.sh start rest -p <port>
 ----
 
-HBase ships a simple REST client, see link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/rest/client/package-summary.html[REST client] package for details.
-To enable SSL support for it, please also import your certificate into local java cacerts keystore:
+To stop the REST server, use Ctrl-C if you were running it in the foreground, or the
+following command if you were running it in the background.
+
+[source, bash]
 ----
-keytool -import -trustcacerts -file /home/user/restserver.cert -keystore $JAVA_HOME/jre/lib/security/cacerts
+$ bin/hbase-daemon.sh stop rest
+----
+
+=== Configuring the REST Server and Client
+
+For information about configuring the REST server and client for SSL, as well as `doAs`
+impersonation for the REST server, see <<security.gateway.thrift>> and other portions
+of the <<security>> chapter.
+
+=== Using REST Endpoints
+
+The following examples use the placeholder server pass:[http://example.com:8000], and
+the following commands can all be run using `curl` or `wget` commands. You can request
+plain text (the default), XML , or JSON output by adding no header for plain text,
+or the header "Accept: text/xml" for XML, "Accept: application/json" for JSON, or
+"Accept: application/x-protobuf" to for protocol buffers.
+
+NOTE: Unless specified, use `GET` requests for queries, `PUT` or `POST` requests for
+creation or mutation, and `DELETE` for deletion.
+
+.Cluster-Wide Endpoints
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/version/cluster
+|GET
+|Version of HBase running on this cluster
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/version/cluster"
+
+|/status/cluster
+|GET
+|Cluster status
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/status/cluster"
+
+|/
+|GET
+|List of all non-system tables
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/"
+
+|===
+
+.Namespace Endpoints
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/namespaces
+|GET
+|List all namespaces
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/namespaces/"
+
+|/namespaces/_namespace_
+|GET
+|Describe a specific namespace
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/namespaces/special_ns"
+
+|/namespaces/_namespace_
+|POST
+|Create a new namespace
+|curl -vi -X POST \
+  -H "Accept: text/xml" \
+  "example.com:8000/namespaces/special_ns"
+
+|/namespaces/_namespace_/tables
+|GET
+|List all tables in a specific namespace
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/namespaces/special_ns/tables"
+
+|/namespaces/_namespace_
+|PUT
+|Alter an existing namespace. Currently not used.
+|curl -vi -X PUT \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/namespaces/special_ns
+
+|/namespaces/_namespace_
+|DELETE
+|Delete a namespace. The namespace must be empty.
+|curl -vi -X DELETE \
+  -H "Accept: text/xml" \
+  "example.com:8000/namespaces/special_ns"
+
+|===
+
+.Table Endpoints
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/_table_/schema
+|GET
+|Describe the schema of the specified table.
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/schema"
+
+|/_table_/schema
+|POST
+|Create a new table, or replace an existing table's schema
+|curl -vi -X POST \
+  -H "Accept: text/xml" \
+  -H "Content-Type: text/xml" \
+  -d '&lt;?xml version="1.0" encoding="UTF-8"?>&lt;TableSchema name="users">&lt;ColumnSchema name="cf" />&lt;/TableSchema>' \
+  "http://example.com:8000/users/schema"
+
+|/_table_/schema
+|PUT
+|Update an existing table with the provided schema fragment
+|curl -vi -X PUT \
+  -H "Accept: text/xml" \
+  -H "Content-Type: text/xml" \
+  -d '&lt;?xml version="1.0" encoding="UTF-8"?>&lt;TableSchema name="users">&lt;ColumnSchema name="cf" KEEP_DELETED_CELLS="true" />&lt;/TableSchema>' \
+  "http://example.com:8000/users/schema"
+
+|/_table_/schema
+|DELETE
+|Delete the table. You must use the `/_table_/schema` endpoint, not just `/_table_/`.
+|curl -vi -X DELETE \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/schema"
+
+|/_table_/regions
+|GET
+|List the table regions
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/regions
+|===
+
+.Endpoints for `Get` Operations
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/_table_/_row_/_column:qualifier_/_timestamp_
+|GET
+|Get the value of a single row. Values are Base-64 encoded.
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/row1"
+
+curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/row1/cf:a/1458586888395"
+
+|/_table_/_row_/_column:qualifier_
+|GET
+|Get the value of a single column. Values are Base-64 encoded.
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/row1/cf:a"
+
+curl -vi -X GET \
+  -H "Accept: text/xml" \
+   "http://example.com:8000/users/row1/cf:a/"
+
+|/_table_/_row_/_column:qualifier_/?v=_number_of_versions_
+|GET
+|Multi-Get a specified number of versions of a given cell. Values are Base-64 encoded.
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/row1/cf:a?v=2"
+
+|===
+
+.Endpoints for `Scan` Operations
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/_table_/scanner/
+|PUT
+|Get a Scanner object. Required by all other Scan operations. Adjust the batch parameter
+to the number of rows the scan should return in a batch. See the next example for
+adding filters to your scanner. The scanner endpoint URL is returned as the `Location`
+in the HTTP response. The other examples in this table assume that the scanner endpoint
+is `\http://example.com:8000/users/scanner/145869072824375522207`.
+|curl -vi -X PUT \
+  -H "Accept: text/xml" \
+  -H "Content-Type: text/xml" \
+  -d '<Scanner batch="1"/>' \
+  "http://example.com:8000/users/scanner/"
+
+|/_table_/scanner/
+|PUT
+|To supply filters to the Scanner object or configure the
+Scanner in any other way, you can create a text file and add
+your filter to the file. For example, to return only rows for
+which keys start with <codeph>u123</codeph> and use a batch size
+of 100, the filter file would look like this:
+
++++
+<pre>
+&lt;Scanner batch="100"&gt;
+  &lt;filter&gt;
+    {
+      "type": "PrefixFilter",
+      "value": "u123"
+    }
+  &lt;/filter&gt;
+&lt;/Scanner&gt;
+</pre>
++++
+
+Pass the file to the `-d` argument of the `curl` request.
+|curl -vi -X PUT \
+  -H "Accept: text/xml" \
+  -H "Content-Type:text/xml" \
+  -d @filter.txt \
+  "http://example.com:8000/users/scanner/"
+
+|/_table_/scanner/_scanner-id_
+|GET
+|Get the next batch from the scanner. Cell values are byte-encoded. If the scanner
+has been exhausted, HTTP status `204` is returned.
+|curl -vi -X GET \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/scanner/145869072824375522207"
+
+|_table_/scanner/_scanner-id_
+|DELETE
+|Deletes the scanner and frees the resources it used.
+|curl -vi -X DELETE \
+  -H "Accept: text/xml" \
+  "http://example.com:8000/users/scanner/145869072824375522207"
+
+|===
+
+.Endpoints for `Put` Operations
+[options="header", cols="2m,m,3d,6l"]
+|===
+|Endpoint
+|HTTP Verb
+|Description
+|Example
+
+|/_table_/_row_key_
+|PUT
+|Write a row to a table. The row, column qualifier, and value must each be Base-64
+encoded. To encode a string, use the `base64` command-line utility. To decode the
+string, use `base64 -d`. The payload is in the `--data` argument, and the `/users/fakerow`
+value is a placeholder. Insert multiple rows by adding them to the `<CellSet>`
+element. You can also save the data to be inserted to a file and pass it to the `-d`
+parameter with syntax like `-d @filename.txt`.
+|curl -vi -X PUT \
+  -H "Accept: text/xml" \
+  -H "Content-Type: text/xml" \
+  -d '<?xml version="1.0" encoding="UTF-8" standalone="yes"?><CellSet><Row key="cm93NQo="><Cell column="Y2Y6ZQo=">dmFsdWU1Cg==</Cell></Row></CellSet>' \
+  "http://example.com:8000/users/fakerow"
+
+curl -vi -X PUT \
+  -H "Accept: text/json" \
+  -H "Content-Type: text/json" \
+  -d '{"Row":[{"key":"cm93NQo=", "Cell": [{"column":"Y2Y6ZQo=", "$":"dmFsdWU1Cg=="}]}]}'' \
+  "example.com:8000/users/fakerow"
+
+|===
+[[xml_schema]]
+=== REST XML Schema
+
+[source,xml]
+----
+<schema xmlns="http://www.w3.org/2001/XMLSchema" xmlns:tns="RESTSchema">
+
+  <element name="Version" type="tns:Version"></element>
+
+  <complexType name="Version">
+    <attribute name="REST" type="string"></attribute>
+    <attribute name="JVM" type="string"></attribute>
+    <attribute name="OS" type="string"></attribute>
+    <attribute name="Server" type="string"></attribute>
+    <attribute name="Jersey" type="string"></attribute>
+  </complexType>
+
+  <element name="TableList" type="tns:TableList"></element>
+
+  <complexType name="TableList">
+    <sequence>
+      <element name="table" type="tns:Table" maxOccurs="unbounded" minOccurs="1"></element>
+    </sequence>
+  </complexType>
+
+  <complexType name="Table">
+    <sequence>
+      <element name="name" type="string"></element>
+    </sequence>
+  </complexType>
+
+  <element name="TableInfo" type="tns:TableInfo"></element>
+
+  <complexType name="TableInfo">
+    <sequence>
+      <element name="region" type="tns:TableRegion" maxOccurs="unbounded" minOccurs="1"></element>
+    </sequence>
+    <attribute name="name" type="string"></attribute>
+  </complexType>
+
+  <complexType name="TableRegion">
+    <attribute name="name" type="string"></attribute>
+    <attribute name="id" type="int"></attribute>
+    <attribute name="startKey" type="base64Binary"></attribute>
+    <attribute name="endKey" type="base64Binary"></attribute>
+    <attribute name="location" type="string"></attribute>
+  </complexType>
+
+  <element name="TableSchema" type="tns:TableSchema"></element>
+
+  <complexType name="TableSchema">
+    <sequence>
+      <element name="column" type="tns:ColumnSchema" maxOccurs="unbounded" minOccurs="1"></element>
+    </sequence>
+    <attribute name="name" type="string"></attribute>
+    <anyAttribute></anyAttribute>
+  </complexType>
+
+  <complexType name="ColumnSchema">
+    <attribute name="name" type="string"></attribute>
+    <anyAttribute></anyAttribute>
+  </complexType>
+
+  <element name="CellSet" type="tns:CellSet"></element>
+
+  <complexType name="CellSet">
+    <sequence>
+      <element name="row" type="tns:Row" maxOccurs="unbounded" minOccurs="1"></element>
+    </sequence>
+  </complexType>
+
+  <element name="Row" type="tns:Row"></element>
+
+  <complexType name="Row">
+    <sequence>
+      <element name="key" type="base64Binary"></element>
+      <element name="cell" type="tns:Cell" maxOccurs="unbounded" minOccurs="1"></element>
+    </sequence>
+  </complexType>
+
+  <element name="Cell" type="tns:Cell"></element>
+
+  <complexType name="Cell">
+    <sequence>
+      <element name="value" maxOccurs="1" minOccurs="1">
+        <simpleType><restriction base="base64Binary">
+        </simpleType>
+      </element>
+    </sequence>
+    <attribute name="column" type="base64Binary" />
+    <attribute name="timestamp" type="int" />
+  </complexType>
+
+  <element name="Scanner" type="tns:Scanner"></element>
+
+  <complexType name="Scanner">
+    <sequence>
+      <element name="column" type="base64Binary" minOccurs="0" maxOccurs="unbounded"></element>
+    </sequence>
+    <sequence>
+      <element name="filter" type="string" minOccurs="0" maxOccurs="1"></element>
+    </sequence>
+    <attribute name="startRow" type="base64Binary"></attribute>
+    <attribute name="endRow" type="base64Binary"></attribute>
+    <attribute name="batch" type="int"></attribute>
+    <attribute name="startTime" type="int"></attribute>
+    <attribute name="endTime" type="int"></attribute>
+  </complexType>
+
+  <element name="StorageClusterVersion" type="tns:StorageClusterVersion" />
+
+  <complexType name="StorageClusterVersion">
+    <attribute name="version" type="string"></attribute>
+  </complexType>
+
+  <element name="StorageClusterStatus"
+    type="tns:StorageClusterStatus">
+  </element>
+
+  <complexType name="StorageClusterStatus">
+    <sequence>
+      <element name="liveNode" type="tns:Node"
+        maxOccurs="unbounded" minOccurs="0">
+      </element>
+      <element name="deadNode" type="string" maxOccurs="unbounded"
+        minOccurs="0">
+      </element>
+    </sequence>
+    <attribute name="regions" type="int"></attribute>
+    <attribute name="requests" type="int"></attribute>
+    <attribute name="averageLoad" type="float"></attribute>
+  </complexType>
+
+  <complexType name="Node">
+    <sequence>
+      <element name="region" type="tns:Region"
+   maxOccurs="unbounded" minOccurs="0">
+      </element>
+    </sequence>
+    <attribute name="name" type="string"></attribute>
+    <attribute name="startCode" type="int"></attribute>
+    <attribute name="requests" type="int"></attribute>
+    <attribute name="heapSizeMB" type="int"></attribute>
+    <attribute name="maxHeapSizeMB" type="int"></attribute>
+  </complexType>
+
+  <complexType name="Region">
+    <attribute name="name" type="base64Binary"></attribute>
+    <attribute name="stores" type="int"></attribute>
+    <attribute name="storefiles" type="int"></attribute>
+    <attribute name="storefileSizeMB" type="int"></attribute>
+    <attribute name="memstoreSizeMB" type="int"></attribute>
+    <attribute name="storefileIndexSizeMB" type="int"></attribute>
+  </complexType>
+
+</schema>
+----
+
+[[protobufs_schema]]
+=== REST Protobufs Schema
+
+[source,json]
+----
+message Version {
+  optional string restVersion = 1;
+  optional string jvmVersion = 2;
+  optional string osVersion = 3;
+  optional string serverVersion = 4;
+  optional string jerseyVersion = 5;
+}
+
+message StorageClusterStatus {
+  message Region {
+    required bytes name = 1;
+    optional int32 stores = 2;
+    optional int32 storefiles = 3;
+    optional int32 storefileSizeMB = 4;
+    optional int32 memstoreSizeMB = 5;
+    optional int32 storefileIndexSizeMB = 6;
+  }
+  message Node {
+    required string name = 1;    // name:port
+    optional int64 startCode = 2;
+    optional int32 requests = 3;
+    optional int32 heapSizeMB = 4;
+    optional int32 maxHeapSizeMB = 5;
+    repeated Region regions = 6;
+  }
+  // node status
+  repeated Node liveNodes = 1;
+  repeated string deadNodes = 2;
+  // summary statistics
+  optional int32 regions = 3;
+  optional int32 requests = 4;
+  optional double averageLoad = 5;
+}
+
+message TableList {
+  repeated string name = 1;
+}
+
+message TableInfo {
+  required string name = 1;
+  message Region {
+    required string name = 1;
+    optional bytes startKey = 2;
+    optional bytes endKey = 3;
+    optional int64 id = 4;
+    optional string location = 5;
+  }
+  repeated Region regions = 2;
+}
+
+message TableSchema {
+  optional string name = 1;
+  message Attribute {
+    required string name = 1;
+    required string value = 2;
+  }
+  repeated Attribute attrs = 2;
+  repeated ColumnSchema columns = 3;
+  // optional helpful encodings of commonly used attributes
+  optional bool inMemory = 4;
+  optional bool readOnly = 5;
+}
+
+message ColumnSchema {
+  optional string name = 1;
+  message Attribute {
+    required string name = 1;
+    required string value = 2;
+  }
+  repeated Attribute attrs = 2;
+  // optional helpful encodings of commonly used attributes
+  optional int32 ttl = 3;
+  optional int32 maxVersions = 4;
+  optional string compression = 5;
+}
+
+message Cell {
+  optional bytes row = 1;       // unused if Cell is in a CellSet
+  optional bytes column = 2;
+  optional int64 timestamp = 3;
+  optional bytes data = 4;
+}
+
+message CellSet {
+  message Row {
+    required bytes key = 1;
+    repeated Cell values = 2;
+  }
+  repeated Row rows = 1;
+}
+
+message Scanner {
+  optional bytes startRow = 1;
+  optional bytes endRow = 2;
+  repeated bytes columns = 3;
+  optional int32 batch = 4;
+  optional int64 startTime = 5;
+  optional int64 endTime = 6;
+}
 ----
 
 == Thrift
@@ -64,3 +624,331 @@ Documentation about Thrift has moved to <<thrift>>.
 
 FB's Chip Turner wrote a pure C/C++ client.
 link:https://github.com/facebook/native-cpp-hbase-client[Check it out].
+
+[[jdo]]
+
+== Using Java Data Objects (JDO) with HBase
+
+link:https://db.apache.org/jdo/[Java Data Objects (JDO)] is a standard way to
+access persistent data in databases, using plain old Java objects (POJO) to
+represent persistent data.
+
+.Dependencies
+This code example has the following dependencies:
+
+. HBase 0.90.x or newer
+. commons-beanutils.jar (http://commons.apache.org/)
+. commons-pool-1.5.5.jar (http://commons.apache.org/)
+. transactional-tableindexed for HBase 0.90 (https://github.com/hbase-trx/hbase-transactional-tableindexed)
+
+.Download `hbase-jdo`
+Download the code from http://code.google.com/p/hbase-jdo/.
+
+.JDO Example
+====
+
+This example uses JDO to create a table and an index, insert a row into a table, get
+a row, get a column value, perform a query, and do some additional HBase operations.
+
+[source, java]
+----
+package com.apache.hadoop.hbase.client.jdo.examples;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Hashtable;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.tableindexed.IndexedTable;
+
+import com.apache.hadoop.hbase.client.jdo.AbstractHBaseDBO;
+import com.apache.hadoop.hbase.client.jdo.HBaseBigFile;
+import com.apache.hadoop.hbase.client.jdo.HBaseDBOImpl;
+import com.apache.hadoop.hbase.client.jdo.query.DeleteQuery;
+import com.apache.hadoop.hbase.client.jdo.query.HBaseOrder;
+import com.apache.hadoop.hbase.client.jdo.query.HBaseParam;
+import com.apache.hadoop.hbase.client.jdo.query.InsertQuery;
+import com.apache.hadoop.hbase.client.jdo.query.QSearch;
+import com.apache.hadoop.hbase.client.jdo.query.SelectQuery;
+import com.apache.hadoop.hbase.client.jdo.query.UpdateQuery;
+
+/**
+ * Hbase JDO Example.
+ *
+ * dependency library.
+ * - commons-beanutils.jar
+ * - commons-pool-1.5.5.jar
+ * - hbase0.90.0-transactionl.jar
+ *
+ * you can expand Delete,Select,Update,Insert Query classes.
+ *
+ */
+public class HBaseExample {
+  public static void main(String[] args) throws Exception {
+    AbstractHBaseDBO dbo = new HBaseDBOImpl();
+
+    //*drop if table is already exist.*
+    if(dbo.isTableExist("user")){
+     dbo.deleteTable("user");
+    }
+
+    //*create table*
+    dbo.createTableIfNotExist("user",HBaseOrder.DESC,"account");
+    //dbo.createTableIfNotExist("user",HBaseOrder.ASC,"account");
+
+    //create index.
+    String[] cols={"id","name"};
+    dbo.addIndexExistingTable("user","account",cols);
+
+    //insert
+    InsertQuery insert = dbo.createInsertQuery("user");
+    UserBean bean = new UserBean();
+    bean.setFamily("account");
+    bean.setAge(20);
+    bean.setEmail("ncanis@gmail.com");
+    bean.setId("ncanis");
+    bean.setName("ncanis");
+    bean.setPassword("1111");
+    insert.insert(bean);
+
+    //select 1 row
+    SelectQuery select = dbo.createSelectQuery("user");
+    UserBean resultBean = (UserBean)select.select(bean.getRow(),UserBean.class);
+
+    // select column value.
+    String value = (String)select.selectColumn(bean.getRow(),"account","id",String.class);
+
+    // search with option (QSearch has EQUAL, NOT_EQUAL, LIKE)
+    // select id,password,name,email from account where id='ncanis' limit startRow,20
+    HBaseParam param = new HBaseParam();
+    param.setPage(bean.getRow(),20);
+    param.addColumn("id","password","name","email");
+    param.addSearchOption("id","ncanis",QSearch.EQUAL);
+    select.search("account", param, UserBean.class);
+
+    // search column value is existing.
+    boolean isExist = select.existColumnValue("account","id","ncanis".getBytes());
+
+    // update password.
+    UpdateQuery update = dbo.createUpdateQuery("user");
+    Hashtable<String, byte[]> colsTable = new Hashtable<String, byte[]>();
+    colsTable.put("password","2222".getBytes());
+    update.update(bean.getRow(),"account",colsTable);
+
+    //delete
+    DeleteQuery delete = dbo.createDeleteQuery("user");
+    delete.deleteRow(resultBean.getRow());
+
+    ////////////////////////////////////
+    // etc
+
+    // HTable pool with apache commons pool
+    // borrow and release. HBasePoolManager(maxActive, minIdle etc..)
+    IndexedTable table = dbo.getPool().borrow("user");
+    dbo.getPool().release(table);
+
+    // upload bigFile by hadoop directly.
+    HBaseBigFile bigFile = new HBaseBigFile();
+    File file = new File("doc/movie.avi");
+    FileInputStream fis = new FileInputStream(file);
+    Path rootPath = new Path("/files/");
+    String filename = "movie.avi";
+    bigFile.uploadFile(rootPath,filename,fis,true);
+
+    // receive file stream from hadoop.
+    Path p = new Path(rootPath,filename);
+    InputStream is = bigFile.path2Stream(p,4096);
+
+  }
+}
+----
+====
+
+[[scala]]
+== Scala
+
+=== Setting the Classpath
+
+To use Scala with HBase, your CLASSPATH must include HBase's classpath as well as
+the Scala JARs required by your code. First, use the following command on a server
+running the HBase RegionServer process, to get HBase's classpath.
+
+[source, bash]
+----
+$ ps aux |grep regionserver| awk -F 'java.library.path=' {'print $2'} | awk {'print $1'}
+
+/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64
+----
+
+Set the `$CLASSPATH` environment variable to include the path you found in the previous
+step, plus the path of `scala-library.jar` and each additional Scala-related JAR needed for
+your project.
+
+[source, bash]
+----
+$ export CLASSPATH=$CLASSPATH:/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64:/path/to/scala-library.jar
+----
+
+=== Scala SBT File
+
+Your `build.sbt` file needs the following `resolvers` and `libraryDependencies` to work
+with HBase.
+
+----
+resolvers += "Apache HBase" at "https://repository.apache.org/content/repositories/releases"
+
+resolvers += "Thrift" at "http://people.apache.org/~rawson/repo/"
+
+libraryDependencies ++= Seq(
+    "org.apache.hadoop" % "hadoop-core" % "0.20.2",
+    "org.apache.hbase" % "hbase" % "0.90.4"
+)
+----
+
+=== Example Scala Code
+
+This example lists HBase tables, creates a new table, and adds a row to it.
+
+[source, scala]
+----
+import org.apache.hadoop.hbase.HBaseConfiguration
+import org.apache.hadoop.hbase.client.{Connection,ConnectionFactory,HBaseAdmin,HTable,Put,Get}
+import org.apache.hadoop.hbase.util.Bytes
+
+
+val conf = new HBaseConfiguration()
+val connection = ConnectionFactory.createConnection(conf);
+val admin = connection.getAdmin();
+
+// list the tables
+val listtables=admin.listTables()
+listtables.foreach(println)
+
+// let's insert some data in 'mytable' and get the row
+
+val table = new HTable(conf, "mytable")
+
+val theput= new Put(Bytes.toBytes("rowkey1"))
+
+theput.add(Bytes.toBytes("ids"),Bytes.toBytes("id1"),Bytes.toBytes("one"))
+table.put(theput)
+
+val theget= new Get(Bytes.toBytes("rowkey1"))
+val result=table.get(theget)
+val value=result.value()
+println(Bytes.toString(value))
+----
+
+[[jython]]
+== Jython
+
+
+=== Setting the Classpath
+
+To use Jython with HBase, your CLASSPATH must include HBase's classpath as well as
+the Jython JARs required by your code. First, use the following command on a server
+running the HBase RegionServer process, to get HBase's classpath.
+
+[source, bash]
+----
+$ ps aux |grep regionserver| awk -F 'java.library.path=' {'print $2'} | awk {'print $1'}
+
+/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64
+----
+
+Set the `$CLASSPATH` environment variable to include the path you found in the previous
+step, plus the path to `jython.jar` and each additional Jython-related JAR needed for
+your project.
+
+[source, bash]
+----
+$ export CLASSPATH=$CLASSPATH:/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64:/path/to/jython.jar
+----
+
+Start a Jython shell with HBase and Hadoop JARs in the classpath:
+$ bin/hbase org.python.util.jython
+
+=== Jython Code Examples
+
+.Table Creation, Population, Get, and Delete with Jython
+====
+The following Jython code example creates a table, populates it with data, fetches
+the data, and deletes the table.
+
+[source,jython]
+----
+import java.lang
+from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants, TableName
+from org.apache.hadoop.hbase.client import HBaseAdmin, HTable, Get
+from org.apache.hadoop.hbase.io import Cell, RowResult
+
+# First get a conf object.  This will read in the configuration
+# that is out in your hbase-*.xml files such as location of the
+# hbase master node.
+conf = HBaseConfiguration()
+
+# Create a table named 'test' that has two column families,
+# one named 'content, and the other 'anchor'.  The colons
+# are required for column family names.
+tablename = TableName.valueOf("test")
+
+desc = HTableDescriptor(tablename)
+desc.addFamily(HColumnDescriptor("content:"))
+desc.addFamily(HColumnDescriptor("anchor:"))
+admin = HBaseAdmin(conf)
+
+# Drop and recreate if it exists
+if admin.tableExists(tablename):
+    admin.disableTable(tablename)
+    admin.deleteTable(tablename)
+admin.createTable(desc)
+
+tables = admin.listTables()
+table = HTable(conf, tablename)
+
+# Add content to 'column:' on a row named 'row_x'
+row = 'row_x'
+update = Get(row)
+update.put('content:', 'some content')
+table.commit(update)
+
+# Now fetch the content just added, returns a byte[]
+data_row = table.get(row, "content:")
+data = java.lang.String(data_row.value, "UTF8")
+
+print "The fetched row contains the value '%s'" % data
+
+# Delete the table.
+admin.disableTable(desc.getName())
+admin.deleteTable(desc.getName())
+----
+====
+
+.Table Scan Using Jython
+====
+This example scans a table and returns the results that match a given family qualifier.
+
+[source, jython]
+----
+# Print all rows that are members of a particular column family
+# by passing a regex for family qualifier
+
+import java.lang
+
+from org.apache.hadoop.hbase import HBaseConfiguration
+from org.apache.hadoop.hbase.client import HTable
+
+conf = HBaseConfiguration()
+
+table = HTable(conf, "wiki")
+col = "title:.*$"
+
+scanner = table.getScanner([col], "")
+while 1:
+    result = scanner.next()
+    if not result:
+        break
+    print java.lang.String(result.row), java.lang.String(result.get('title:').value)
+----
+====

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/faq.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/faq.adoc b/src/main/asciidoc/_chapters/faq.adoc
index 22e4ad3..7bffe0e 100644
--- a/src/main/asciidoc/_chapters/faq.adoc
+++ b/src/main/asciidoc/_chapters/faq.adoc
@@ -46,7 +46,7 @@ What is the history of HBase?::
 
 === Upgrading
 How do I upgrade Maven-managed projects from HBase 0.94 to HBase 0.96+?::
-  In HBase 0.96, the project moved to a modular structure. Adjust your project's dependencies to rely upon the `hbase-client` module or another module as appropriate, rather than a single JAR. You can model your Maven depency after one of the following, depending on your targeted version of HBase. See Section 3.5, \u201cUpgrading from 0.94.x to 0.96.x\u201d or Section 3.3, \u201cUpgrading from 0.96.x to 0.98.x\u201d for more information.
+  In HBase 0.96, the project moved to a modular structure. Adjust your project's dependencies to rely upon the `hbase-client` module or another module as appropriate, rather than a single JAR. You can model your Maven dependency after one of the following, depending on your targeted version of HBase. See Section 3.5, \u201cUpgrading from 0.94.x to 0.96.x\u201d or Section 3.3, \u201cUpgrading from 0.96.x to 0.98.x\u201d for more information.
 +
 .Maven Dependency for HBase 0.98
 [source,xml]
@@ -55,18 +55,18 @@ How do I upgrade Maven-managed projects from HBase 0.94 to HBase 0.96+?::
   <groupId>org.apache.hbase</groupId>
   <artifactId>hbase-client</artifactId>
   <version>0.98.5-hadoop2</version>
-</dependency>  
-----              
-+    
-.Maven Dependency for HBase 0.96       
+</dependency>
+----
++
+.Maven Dependency for HBase 0.96
 [source,xml]
 ----
 <dependency>
   <groupId>org.apache.hbase</groupId>
   <artifactId>hbase-client</artifactId>
   <version>0.96.2-hadoop2</version>
-</dependency>  
-----           
+</dependency>
+----
 +
 .Maven Dependency for HBase 0.94
 [source,xml]
@@ -75,9 +75,9 @@ How do I upgrade Maven-managed projects from HBase 0.94 to HBase 0.96+?::
   <groupId>org.apache.hbase</groupId>
   <artifactId>hbase</artifactId>
   <version>0.94.3</version>
-</dependency>   
-----         
-                
+</dependency>
+----
+
 
 === Architecture
 How does HBase handle Region-RegionServer assignment and locality?::
@@ -91,7 +91,7 @@ Where can I learn about the rest of the configuration options?::
   See <<configuration>>.
 
 === Schema Design / Data Access
-  
+
 How should I design my schema in HBase?::
   See <<datamodel>> and <<schema>>.
 
@@ -105,7 +105,7 @@ Can I change a table's rowkeys?::
   This is a very common question. You can't. See <<changing.rowkeys>>.
 
 What APIs does HBase support?::
-  See <<datamodel>>, <<architecture.client>>, and <<nonjava.jvm>>.
+  See <<datamodel>>, <<architecture.client>>, and <<external_apis>>.
 
 === MapReduce
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/getting_started.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/getting_started.adoc b/src/main/asciidoc/_chapters/getting_started.adoc
index 41674a0..26af568 100644
--- a/src/main/asciidoc/_chapters/getting_started.adoc
+++ b/src/main/asciidoc/_chapters/getting_started.adoc
@@ -19,6 +19,7 @@
  */
 ////
 
+[[getting_started]]
 = Getting Started
 :doctype: book
 :numbered:
@@ -57,7 +58,7 @@ Prior to HBase 0.94.x, HBase expected the loopback IP address to be 127.0.0.1. U
 
 .Example /etc/hosts File for Ubuntu
 ====
-The following _/etc/hosts_ file works correctly for HBase 0.94.x and earlier, on Ubuntu. Use this as a template if you run into trouble. 
+The following _/etc/hosts_ file works correctly for HBase 0.94.x and earlier, on Ubuntu. Use this as a template if you run into trouble.
 [listing]
 ----
 127.0.0.1 localhost
@@ -80,15 +81,17 @@ See <<java,Java>> for information about supported JDK versions.
   This will take you to a mirror of _HBase
   Releases_.
   Click on the folder named _stable_ and then download the binary file that ends in _.tar.gz_ to your local filesystem.
-  Be sure to choose the version that corresponds with the version of Hadoop you are likely to use later.
-  In most cases, you should choose the file for Hadoop 2, which will be called something like _hbase-0.98.3-hadoop2-bin.tar.gz_.
+  Prior to 1.x version, be sure to choose the version that corresponds with the version of Hadoop you are
+  likely to use later (in most cases, you should choose the file for Hadoop 2, which will be called
+  something like _hbase-0.98.13-hadoop2-bin.tar.gz_).
   Do not download the file ending in _src.tar.gz_ for now.
 . Extract the downloaded file, and change to the newly-created directory.
 +
+[source,subs="attributes"]
 ----
 
-$ tar xzvf hbase-<?eval ${project.version}?>-hadoop2-bin.tar.gz
-$ cd hbase-<?eval ${project.version}?>-hadoop2/
+$ tar xzvf hbase-{Version}-bin.tar.gz
+$ cd hbase-{Version}/
 ----
 
 . For HBase 0.98.5 and later, you are required to set the `JAVA_HOME` environment variable before starting HBase.
@@ -286,7 +289,7 @@ $
 === Intermediate - Pseudo-Distributed Local Install
 
 After working your way through <<quickstart,quickstart>>, you can re-configure HBase to run in pseudo-distributed mode.
-Pseudo-distributed mode means that HBase still runs completely on a single host, but each HBase daemon (HMaster, HRegionServer, and Zookeeper) runs as a separate process.
+Pseudo-distributed mode means that HBase still runs completely on a single host, but each HBase daemon (HMaster, HRegionServer, and ZooKeeper) runs as a separate process.
 By default, unless you configure the `hbase.rootdir` property as described in <<quickstart,quickstart>>, your data is still stored in _/tmp/_.
 In this walk-through, we store your data in HDFS instead, assuming you have HDFS available.
 You can skip the HDFS configuration to continue storing your data in the local filesystem.
@@ -294,9 +297,11 @@ You can skip the HDFS configuration to continue storing your data in the local f
 .Hadoop Configuration
 [NOTE]
 ====
-This procedure assumes that you have configured Hadoop and HDFS on your local system and or a remote system, and that they are running and available.
-It also assumes you are using Hadoop 2.
-Currently, the documentation on the Hadoop website does not include a quick start for Hadoop 2, but the guide at link:http://www.alexjf.net/blog/distributed-systems/hadoop-yarn-installation-definitive-guide          is a good starting point.
+This procedure assumes that you have configured Hadoop and HDFS on your local system and/or a remote
+system, and that they are running and available. It also assumes you are using Hadoop 2.
+The guide on
+link:http://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/SingleCluster.html[Setting up a Single Node Cluster]
+in the Hadoop documentation is a good starting point.
 ====
 
 
@@ -425,7 +430,7 @@ You can stop HBase the same way as in the <<quickstart,quickstart>> procedure, u
 
 In reality, you need a fully-distributed configuration to fully test HBase and to use it in real-world scenarios.
 In a distributed configuration, the cluster contains multiple nodes, each of which runs one or more HBase daemon.
-These include primary and backup Master instances, multiple Zookeeper nodes, and multiple RegionServer nodes.
+These include primary and backup Master instances, multiple ZooKeeper nodes, and multiple RegionServer nodes.
 
 This advanced quickstart adds two more nodes to your cluster.
 The architecture will be as follows:

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/hbase-default.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/hbase-default.adoc b/src/main/asciidoc/_chapters/hbase-default.adoc
index bf56dd3..60c0849 100644
--- a/src/main/asciidoc/_chapters/hbase-default.adoc
+++ b/src/main/asciidoc/_chapters/hbase-default.adoc
@@ -46,7 +46,7 @@ Temporary directory on the local filesystem.
 .Default
 `${java.io.tmpdir}/hbase-${user.name}`
 
-  
+
 [[hbase.rootdir]]
 *`hbase.rootdir`*::
 +
@@ -64,7 +64,7 @@ The directory shared by region servers and into
 .Default
 `${hbase.tmp.dir}/hbase`
 
-  
+
 [[hbase.cluster.distributed]]
 *`hbase.cluster.distributed`*::
 +
@@ -77,7 +77,7 @@ The mode the cluster will be in. Possible values are
 .Default
 `false`
 
-  
+
 [[hbase.zookeeper.quorum]]
 *`hbase.zookeeper.quorum`*::
 +
@@ -97,7 +97,7 @@ Comma separated list of servers in the ZooKeeper ensemble
 .Default
 `localhost`
 
-  
+
 [[hbase.local.dir]]
 *`hbase.local.dir`*::
 +
@@ -108,7 +108,7 @@ Directory on the local filesystem to be used
 .Default
 `${hbase.tmp.dir}/local/`
 
-  
+
 [[hbase.master.info.port]]
 *`hbase.master.info.port`*::
 +
@@ -119,18 +119,18 @@ The port for the HBase Master web UI.
 .Default
 `16010`
 
-  
+
 [[hbase.master.info.bindAddress]]
 *`hbase.master.info.bindAddress`*::
 +
 .Description
 The bind address for the HBase Master web UI
-    
+
 +
 .Default
 `0.0.0.0`
 
-  
+
 [[hbase.master.logcleaner.plugins]]
 *`hbase.master.logcleaner.plugins`*::
 +
@@ -145,7 +145,7 @@ A comma-separated list of BaseLogCleanerDelegate invoked by
 .Default
 `org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner`
 
-  
+
 [[hbase.master.logcleaner.ttl]]
 *`hbase.master.logcleaner.ttl`*::
 +
@@ -156,7 +156,7 @@ Maximum time a WAL can stay in the .oldlogdir directory,
 .Default
 `600000`
 
-  
+
 [[hbase.master.hfilecleaner.plugins]]
 *`hbase.master.hfilecleaner.plugins`*::
 +
@@ -172,18 +172,7 @@ A comma-separated list of BaseHFileCleanerDelegate invoked by
 .Default
 `org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner`
 
-  
-[[hbase.master.catalog.timeout]]
-*`hbase.master.catalog.timeout`*::
-+
-.Description
-Timeout value for the Catalog Janitor from the master to
-    META.
-+
-.Default
-`600000`
 
-  
 [[hbase.master.infoserver.redirect]]
 *`hbase.master.infoserver.redirect`*::
 +
@@ -195,7 +184,7 @@ Whether or not the Master listens to the Master web
 .Default
 `true`
 
-  
+
 [[hbase.regionserver.port]]
 *`hbase.regionserver.port`*::
 +
@@ -205,7 +194,7 @@ The port the HBase RegionServer binds to.
 .Default
 `16020`
 
-  
+
 [[hbase.regionserver.info.port]]
 *`hbase.regionserver.info.port`*::
 +
@@ -216,7 +205,7 @@ The port for the HBase RegionServer web UI
 .Default
 `16030`
 
-  
+
 [[hbase.regionserver.info.bindAddress]]
 *`hbase.regionserver.info.bindAddress`*::
 +
@@ -226,7 +215,7 @@ The address for the HBase RegionServer web UI
 .Default
 `0.0.0.0`
 
-  
+
 [[hbase.regionserver.info.port.auto]]
 *`hbase.regionserver.info.port.auto`*::
 +
@@ -239,7 +228,7 @@ Whether or not the Master or RegionServer
 .Default
 `false`
 
-  
+
 [[hbase.regionserver.handler.count]]
 *`hbase.regionserver.handler.count`*::
 +
@@ -250,7 +239,7 @@ Count of RPC Listener instances spun up on RegionServers.
 .Default
 `30`
 
-  
+
 [[hbase.ipc.server.callqueue.handler.factor]]
 *`hbase.ipc.server.callqueue.handler.factor`*::
 +
@@ -262,7 +251,7 @@ Factor to determine the number of call queues.
 .Default
 `0.1`
 
-  
+
 [[hbase.ipc.server.callqueue.read.ratio]]
 *`hbase.ipc.server.callqueue.read.ratio`*::
 +
@@ -287,12 +276,12 @@ Split the call queues into read and write queues.
       and 2 queues will contain only write requests.
       a read.ratio of 1 means that: 9 queues will contain only read requests
       and 1 queues will contain only write requests.
-    
+
 +
 .Default
 `0`
 
-  
+
 [[hbase.ipc.server.callqueue.scan.ratio]]
 *`hbase.ipc.server.callqueue.scan.ratio`*::
 +
@@ -313,12 +302,12 @@ Given the number of read call queues, calculated from the total number
       and 4 queues will contain only short-read requests.
       a scan.ratio of 0.8 means that: 6 queues will contain only long-read requests
       and 2 queues will contain only short-read requests.
-    
+
 +
 .Default
 `0`
 
-  
+
 [[hbase.regionserver.msginterval]]
 *`hbase.regionserver.msginterval`*::
 +
@@ -329,7 +318,7 @@ Interval between messages from the RegionServer to Master
 .Default
 `3000`
 
-  
+
 [[hbase.regionserver.regionSplitLimit]]
 *`hbase.regionserver.regionSplitLimit`*::
 +
@@ -342,7 +331,7 @@ Limit for the number of regions after which no more region
 .Default
 `2147483647`
 
-  
+
 [[hbase.regionserver.logroll.period]]
 *`hbase.regionserver.logroll.period`*::
 +
@@ -353,7 +342,7 @@ Period at which we will roll the commit log regardless
 .Default
 `3600000`
 
-  
+
 [[hbase.regionserver.logroll.errors.tolerated]]
 *`hbase.regionserver.logroll.errors.tolerated`*::
 +
@@ -367,7 +356,7 @@ The number of consecutive WAL close errors we will allow
 .Default
 `2`
 
-  
+
 [[hbase.regionserver.hlog.reader.impl]]
 *`hbase.regionserver.hlog.reader.impl`*::
 +
@@ -377,7 +366,7 @@ The WAL file reader implementation.
 .Default
 `org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader`
 
-  
+
 [[hbase.regionserver.hlog.writer.impl]]
 *`hbase.regionserver.hlog.writer.impl`*::
 +
@@ -387,7 +376,7 @@ The WAL file writer implementation.
 .Default
 `org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter`
 
-  
+
 [[hbase.master.distributed.log.replay]]
 *`hbase.master.distributed.log.replay`*::
 +
@@ -397,13 +386,13 @@ Enable 'distributed log replay' as default engine splitting
     back to the old mode 'distributed log splitter', set the value to
     'false'.  'Disributed log replay' improves MTTR because it does not
     write intermediate files.  'DLR' required that 'hfile.format.version'
-    be set to version 3 or higher. 
-    
+    be set to version 3 or higher.
+
 +
 .Default
 `true`
 
-  
+
 [[hbase.regionserver.global.memstore.size]]
 *`hbase.regionserver.global.memstore.size`*::
 +
@@ -416,20 +405,20 @@ Maximum size of all memstores in a region server before new
 .Default
 `0.4`
 
-  
+
 [[hbase.regionserver.global.memstore.size.lower.limit]]
 *`hbase.regionserver.global.memstore.size.lower.limit`*::
 +
 .Description
 Maximum size of all memstores in a region server before flushes are forced.
       Defaults to 95% of hbase.regionserver.global.memstore.size.
-      A 100% value for this value causes the minimum possible flushing to occur when updates are 
+      A 100% value for this value causes the minimum possible flushing to occur when updates are
       blocked due to memstore limiting.
 +
 .Default
 `0.95`
 
-  
+
 [[hbase.regionserver.optionalcacheflushinterval]]
 *`hbase.regionserver.optionalcacheflushinterval`*::
 +
@@ -441,17 +430,7 @@ Maximum size of all memstores in a region server before flushes are forced.
 .Default
 `3600000`
 
-  
-[[hbase.regionserver.catalog.timeout]]
-*`hbase.regionserver.catalog.timeout`*::
-+
-.Description
-Timeout value for the Catalog Janitor from the regionserver to META.
-+
-.Default
-`600000`
 
-  
 [[hbase.regionserver.dns.interface]]
 *`hbase.regionserver.dns.interface`*::
 +
@@ -462,7 +441,7 @@ The name of the Network Interface from which a region server
 .Default
 `default`
 
-  
+
 [[hbase.regionserver.dns.nameserver]]
 *`hbase.regionserver.dns.nameserver`*::
 +
@@ -474,7 +453,7 @@ The host name or IP address of the name server (DNS)
 .Default
 `default`
 
-  
+
 [[hbase.regionserver.region.split.policy]]
 *`hbase.regionserver.region.split.policy`*::
 +
@@ -483,12 +462,12 @@ The host name or IP address of the name server (DNS)
       A split policy determines when a region should be split. The various other split policies that
       are available currently are ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
       DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy etc.
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy`
 
-  
+
 [[zookeeper.session.timeout]]
 *`zookeeper.session.timeout`*::
 +
@@ -497,17 +476,18 @@ ZooKeeper session timeout in milliseconds. It is used in two different ways.
       First, this value is used in the ZK client that HBase uses to connect to the ensemble.
       It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See
       http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
-      For example, if a HBase region server connects to a ZK ensemble that's also managed by HBase, then the
+      For example, if an HBase region server connects to a ZK ensemble that's also managed
+      by HBase, then the
       session timeout will be the one specified by this configuration. But, a region server that connects
       to an ensemble managed with a different configuration will be subjected that ensemble's maxSessionTimeout. So,
       even though HBase might propose using 90 seconds, the ensemble can have a max timeout lower than this and
       it will take precedence. The current default that ZK ships with is 40 seconds, which is lower than HBase's.
-    
+
 +
 .Default
 `90000`
 
-  
+
 [[zookeeper.znode.parent]]
 *`zookeeper.znode.parent`*::
 +
@@ -520,20 +500,7 @@ Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper
 .Default
 `/hbase`
 
-  
-[[zookeeper.znode.rootserver]]
-*`zookeeper.znode.rootserver`*::
-+
-.Description
-Path to ZNode holding root region location. This is written by
-      the master and read by clients and region servers. If a relative path is
-      given, the parent folder will be ${zookeeper.znode.parent}. By default,
-      this means the root location is stored at /hbase/root-region-server.
-+
-.Default
-`root-region-server`
 
-  
 [[zookeeper.znode.acl.parent]]
 *`zookeeper.znode.acl.parent`*::
 +
@@ -543,7 +510,7 @@ Root ZNode for access control lists.
 .Default
 `acl`
 
-  
+
 [[hbase.zookeeper.dns.interface]]
 *`hbase.zookeeper.dns.interface`*::
 +
@@ -554,7 +521,7 @@ The name of the Network Interface from which a ZooKeeper server
 .Default
 `default`
 
-  
+
 [[hbase.zookeeper.dns.nameserver]]
 *`hbase.zookeeper.dns.nameserver`*::
 +
@@ -566,7 +533,7 @@ The host name or IP address of the name server (DNS)
 .Default
 `default`
 
-  
+
 [[hbase.zookeeper.peerport]]
 *`hbase.zookeeper.peerport`*::
 +
@@ -578,7 +545,7 @@ Port used by ZooKeeper peers to talk to each other.
 .Default
 `2888`
 
-  
+
 [[hbase.zookeeper.leaderport]]
 *`hbase.zookeeper.leaderport`*::
 +
@@ -590,7 +557,7 @@ Port used by ZooKeeper for leader election.
 .Default
 `3888`
 
-  
+
 [[hbase.zookeeper.useMulti]]
 *`hbase.zookeeper.useMulti`*::
 +
@@ -605,21 +572,7 @@ Instructs HBase to make use of ZooKeeper's multi-update functionality.
 .Default
 `true`
 
-  
-[[hbase.config.read.zookeeper.config]]
-*`hbase.config.read.zookeeper.config`*::
-+
-.Description
 
-        Set to true to allow HBaseConfiguration to read the
-        zoo.cfg file for ZooKeeper properties. Switching this to true
-        is not recommended, since the functionality of reading ZK
-        properties from a zoo.cfg file has been deprecated.
-+
-.Default
-`false`
-
-  
 [[hbase.zookeeper.property.initLimit]]
 *`hbase.zookeeper.property.initLimit`*::
 +
@@ -630,7 +583,7 @@ Property from ZooKeeper's config zoo.cfg.
 .Default
 `10`
 
-  
+
 [[hbase.zookeeper.property.syncLimit]]
 *`hbase.zookeeper.property.syncLimit`*::
 +
@@ -642,7 +595,7 @@ Property from ZooKeeper's config zoo.cfg.
 .Default
 `5`
 
-  
+
 [[hbase.zookeeper.property.dataDir]]
 *`hbase.zookeeper.property.dataDir`*::
 +
@@ -653,7 +606,7 @@ Property from ZooKeeper's config zoo.cfg.
 .Default
 `${hbase.tmp.dir}/zookeeper`
 
-  
+
 [[hbase.zookeeper.property.clientPort]]
 *`hbase.zookeeper.property.clientPort`*::
 +
@@ -664,7 +617,7 @@ Property from ZooKeeper's config zoo.cfg.
 .Default
 `2181`
 
-  
+
 [[hbase.zookeeper.property.maxClientCnxns]]
 *`hbase.zookeeper.property.maxClientCnxns`*::
 +
@@ -678,7 +631,7 @@ Property from ZooKeeper's config zoo.cfg.
 .Default
 `300`
 
-  
+
 [[hbase.client.write.buffer]]
 *`hbase.client.write.buffer`*::
 +
@@ -693,7 +646,7 @@ Default size of the HTable client write buffer in bytes.
 .Default
 `2097152`
 
-  
+
 [[hbase.client.pause]]
 *`hbase.client.pause`*::
 +
@@ -706,7 +659,7 @@ General client pause value.  Used mostly as value to wait
 .Default
 `100`
 
-  
+
 [[hbase.client.retries.number]]
 *`hbase.client.retries.number`*::
 +
@@ -721,7 +674,7 @@ Maximum retries.  Used as maximum for all retryable
 .Default
 `35`
 
-  
+
 [[hbase.client.max.total.tasks]]
 *`hbase.client.max.total.tasks`*::
 +
@@ -732,7 +685,7 @@ The maximum number of concurrent tasks a single HTable instance will
 .Default
 `100`
 
-  
+
 [[hbase.client.max.perserver.tasks]]
 *`hbase.client.max.perserver.tasks`*::
 +
@@ -743,7 +696,7 @@ The maximum number of concurrent tasks a single HTable instance will
 .Default
 `5`
 
-  
+
 [[hbase.client.max.perregion.tasks]]
 *`hbase.client.max.perregion.tasks`*::
 +
@@ -756,7 +709,7 @@ The maximum number of concurrent connections the client will
 .Default
 `1`
 
-  
+
 [[hbase.client.scanner.caching]]
 *`hbase.client.scanner.caching`*::
 +
@@ -771,7 +724,7 @@ Number of rows that will be fetched when calling next
 .Default
 `100`
 
-  
+
 [[hbase.client.keyvalue.maxsize]]
 *`hbase.client.keyvalue.maxsize`*::
 +
@@ -786,7 +739,7 @@ Specifies the combined maximum allowed size of a KeyValue
 .Default
 `10485760`
 
-  
+
 [[hbase.client.scanner.timeout.period]]
 *`hbase.client.scanner.timeout.period`*::
 +
@@ -796,7 +749,7 @@ Client scanner lease period in milliseconds.
 .Default
 `60000`
 
-  
+
 [[hbase.client.localityCheck.threadPoolSize]]
 *`hbase.client.localityCheck.threadPoolSize`*::
 +
@@ -806,7 +759,7 @@ Client scanner lease period in milliseconds.
 .Default
 `2`
 
-  
+
 [[hbase.bulkload.retries.number]]
 *`hbase.bulkload.retries.number`*::
 +
@@ -818,7 +771,7 @@ Maximum retries.  This is maximum number of iterations
 .Default
 `10`
 
-  
+
 [[hbase.balancer.period
     ]]
 *`hbase.balancer.period
@@ -830,7 +783,7 @@ Period at which the region balancer runs in the Master.
 .Default
 `300000`
 
-  
+
 [[hbase.regions.slop]]
 *`hbase.regions.slop`*::
 +
@@ -840,7 +793,7 @@ Rebalance if any regionserver has average + (average * slop) regions.
 .Default
 `0.2`
 
-  
+
 [[hbase.server.thread.wakefrequency]]
 *`hbase.server.thread.wakefrequency`*::
 +
@@ -851,20 +804,20 @@ Time to sleep in between searches for work (in milliseconds).
 .Default
 `10000`
 
-  
+
 [[hbase.server.versionfile.writeattempts]]
 *`hbase.server.versionfile.writeattempts`*::
 +
 .Description
 
     How many time to retry attempting to write a version file
-    before just aborting. Each attempt is seperated by the
+    before just aborting. Each attempt is separated by the
     hbase.server.thread.wakefrequency milliseconds.
 +
 .Default
 `3`
 
-  
+
 [[hbase.hregion.memstore.flush.size]]
 *`hbase.hregion.memstore.flush.size`*::
 +
@@ -877,7 +830,7 @@ Time to sleep in between searches for work (in milliseconds).
 .Default
 `134217728`
 
-  
+
 [[hbase.hregion.percolumnfamilyflush.size.lower.bound]]
 *`hbase.hregion.percolumnfamilyflush.size.lower.bound`*::
 +
@@ -890,12 +843,12 @@ Time to sleep in between searches for work (in milliseconds).
     memstore size more than this, all the memstores will be flushed
     (just as usual). This value should be less than half of the total memstore
     threshold (hbase.hregion.memstore.flush.size).
-    
+
 +
 .Default
 `16777216`
 
-  
+
 [[hbase.hregion.preclose.flush.size]]
 *`hbase.hregion.preclose.flush.size`*::
 +
@@ -914,7 +867,7 @@ Time to sleep in between searches for work (in milliseconds).
 .Default
 `5242880`
 
-  
+
 [[hbase.hregion.memstore.block.multiplier]]
 *`hbase.hregion.memstore.block.multiplier`*::
 +
@@ -930,7 +883,7 @@ Time to sleep in between searches for work (in milliseconds).
 .Default
 `4`
 
-  
+
 [[hbase.hregion.memstore.mslab.enabled]]
 *`hbase.hregion.memstore.mslab.enabled`*::
 +
@@ -944,19 +897,19 @@ Time to sleep in between searches for work (in milliseconds).
 .Default
 `true`
 
-  
+
 [[hbase.hregion.max.filesize]]
 *`hbase.hregion.max.filesize`*::
 +
 .Description
 
-    Maximum HFile size. If the sum of the sizes of a region's HFiles has grown to exceed this 
+    Maximum HFile size. If the sum of the sizes of a region's HFiles has grown to exceed this
     value, the region is split in two.
 +
 .Default
 `10737418240`
 
-  
+
 [[hbase.hregion.majorcompaction]]
 *`hbase.hregion.majorcompaction`*::
 +
@@ -973,7 +926,7 @@ Time between major compactions, expressed in milliseconds. Set to 0 to disable
 .Default
 `604800000`
 
-  
+
 [[hbase.hregion.majorcompaction.jitter]]
 *`hbase.hregion.majorcompaction.jitter`*::
 +
@@ -986,32 +939,32 @@ A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occ
 .Default
 `0.50`
 
-  
+
 [[hbase.hstore.compactionThreshold]]
 *`hbase.hstore.compactionThreshold`*::
 +
 .Description
- If more than this number of StoreFiles exist in any one Store 
-      (one StoreFile is written per flush of MemStore), a compaction is run to rewrite all 
+ If more than this number of StoreFiles exist in any one Store
+      (one StoreFile is written per flush of MemStore), a compaction is run to rewrite all
       StoreFiles into a single StoreFile. Larger values delay compaction, but when compaction does
       occur, it takes longer to complete.
 +
 .Default
 `3`
 
-  
+
 [[hbase.hstore.flusher.count]]
 *`hbase.hstore.flusher.count`*::
 +
 .Description
  The number of flush threads. With fewer threads, the MemStore flushes will be
       queued. With more threads, the flushes will be executed in parallel, increasing the load on
-      HDFS, and potentially causing more compactions. 
+      HDFS, and potentially causing more compactions.
 +
 .Default
 `2`
 
-  
+
 [[hbase.hstore.blockingStoreFiles]]
 *`hbase.hstore.blockingStoreFiles`*::
 +
@@ -1023,40 +976,40 @@ A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occ
 .Default
 `10`
 
-  
+
 [[hbase.hstore.blockingWaitTime]]
 *`hbase.hstore.blockingWaitTime`*::
 +
 .Description
  The time for which a region will block updates after reaching the StoreFile limit
-    defined by hbase.hstore.blockingStoreFiles. After this time has elapsed, the region will stop 
+    defined by hbase.hstore.blockingStoreFiles. After this time has elapsed, the region will stop
     blocking updates even if a compaction has not been completed.
 +
 .Default
 `90000`
 
-  
+
 [[hbase.hstore.compaction.min]]
 *`hbase.hstore.compaction.min`*::
 +
 .Description
-The minimum number of StoreFiles which must be eligible for compaction before 
-      compaction can run. The goal of tuning hbase.hstore.compaction.min is to avoid ending up with 
-      too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction 
+The minimum number of StoreFiles which must be eligible for compaction before
+      compaction can run. The goal of tuning hbase.hstore.compaction.min is to avoid ending up with
+      too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction
       each time you have two StoreFiles in a Store, and this is probably not appropriate. If you
-      set this value too high, all the other values will need to be adjusted accordingly. For most 
+      set this value too high, all the other values will need to be adjusted accordingly. For most
       cases, the default value is appropriate. In previous versions of HBase, the parameter
       hbase.hstore.compaction.min was named hbase.hstore.compactionThreshold.
 +
 .Default
 `3`
 
-  
+
 [[hbase.hstore.compaction.max]]
 *`hbase.hstore.compaction.max`*::
 +
 .Description
-The maximum number of StoreFiles which will be selected for a single minor 
+The maximum number of StoreFiles which will be selected for a single minor
       compaction, regardless of the number of eligible StoreFiles. Effectively, the value of
       hbase.hstore.compaction.max controls the length of time it takes a single compaction to
       complete. Setting it larger means that more StoreFiles are included in a compaction. For most
@@ -1065,88 +1018,88 @@ The maximum number of StoreFiles which will be selected for a single minor
 .Default
 `10`
 
-  
+
 [[hbase.hstore.compaction.min.size]]
 *`hbase.hstore.compaction.min.size`*::
 +
 .Description
-A StoreFile smaller than this size will always be eligible for minor compaction. 
-      HFiles this size or larger are evaluated by hbase.hstore.compaction.ratio to determine if 
-      they are eligible. Because this limit represents the "automatic include"limit for all 
-      StoreFiles smaller than this value, this value may need to be reduced in write-heavy 
-      environments where many StoreFiles in the 1-2 MB range are being flushed, because every 
+A StoreFile smaller than this size will always be eligible for minor compaction.
+      HFiles this size or larger are evaluated by hbase.hstore.compaction.ratio to determine if
+      they are eligible. Because this limit represents the "automatic include"limit for all
+      StoreFiles smaller than this value, this value may need to be reduced in write-heavy
+      environments where many StoreFiles in the 1-2 MB range are being flushed, because every
       StoreFile will be targeted for compaction and the resulting StoreFiles may still be under the
       minimum size and require further compaction. If this parameter is lowered, the ratio check is
-      triggered more quickly. This addressed some issues seen in earlier versions of HBase but 
-      changing this parameter is no longer necessary in most situations. Default: 128 MB expressed 
+      triggered more quickly. This addressed some issues seen in earlier versions of HBase but
+      changing this parameter is no longer necessary in most situations. Default: 128 MB expressed
       in bytes.
 +
 .Default
 `134217728`
 
-  
+
 [[hbase.hstore.compaction.max.size]]
 *`hbase.hstore.compaction.max.size`*::
 +
 .Description
-A StoreFile larger than this size will be excluded from compaction. The effect of 
-      raising hbase.hstore.compaction.max.size is fewer, larger StoreFiles that do not get 
+A StoreFile larger than this size will be excluded from compaction. The effect of
+      raising hbase.hstore.compaction.max.size is fewer, larger StoreFiles that do not get
       compacted often. If you feel that compaction is happening too often without much benefit, you
       can try raising this value. Default: the value of LONG.MAX_VALUE, expressed in bytes.
 +
 .Default
 `9223372036854775807`
 
-  
+
 [[hbase.hstore.compaction.ratio]]
 *`hbase.hstore.compaction.ratio`*::
 +
 .Description
-For minor compaction, this ratio is used to determine whether a given StoreFile 
+For minor compaction, this ratio is used to determine whether a given StoreFile
       which is larger than hbase.hstore.compaction.min.size is eligible for compaction. Its
       effect is to limit compaction of large StoreFiles. The value of hbase.hstore.compaction.ratio
-      is expressed as a floating-point decimal. A large ratio, such as 10, will produce a single 
-      giant StoreFile. Conversely, a low value, such as .25, will produce behavior similar to the 
+      is expressed as a floating-point decimal. A large ratio, such as 10, will produce a single
+      giant StoreFile. Conversely, a low value, such as .25, will produce behavior similar to the
       BigTable compaction algorithm, producing four StoreFiles. A moderate value of between 1.0 and
-      1.4 is recommended. When tuning this value, you are balancing write costs with read costs. 
-      Raising the value (to something like 1.4) will have more write costs, because you will 
-      compact larger StoreFiles. However, during reads, HBase will need to seek through fewer 
-      StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of 
-      Bloom filters. Otherwise, you can lower this value to something like 1.0 to reduce the 
-      background cost of writes, and use Bloom filters to control the number of StoreFiles touched 
+      1.4 is recommended. When tuning this value, you are balancing write costs with read costs.
+      Raising the value (to something like 1.4) will have more write costs, because you will
+      compact larger StoreFiles. However, during reads, HBase will need to seek through fewer
+      StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of
+      Bloom filters. Otherwise, you can lower this value to something like 1.0 to reduce the
+      background cost of writes, and use Bloom filters to control the number of StoreFiles touched
       during reads. For most cases, the default value is appropriate.
 +
 .Default
 `1.2F`
 
-  
+
 [[hbase.hstore.compaction.ratio.offpeak]]
 *`hbase.hstore.compaction.ratio.offpeak`*::
 +
 .Description
 Allows you to set a different (by default, more aggressive) ratio for determining
-      whether larger StoreFiles are included in compactions during off-peak hours. Works in the 
-      same way as hbase.hstore.compaction.ratio. Only applies if hbase.offpeak.start.hour and 
+      whether larger StoreFiles are included in compactions during off-peak hours. Works in the
+      same way as hbase.hstore.compaction.ratio. Only applies if hbase.offpeak.start.hour and
       hbase.offpeak.end.hour are also enabled.
 +
 .Default
 `5.0F`
 
-  
+
 [[hbase.hstore.time.to.purge.deletes]]
 *`hbase.hstore.time.to.purge.deletes`*::
 +
 .Description
-The amount of time to delay purging of delete markers with future timestamps. If 
-      unset, or set to 0, all delete markers, including those with future timestamps, are purged 
-      during the next major compaction. Otherwise, a delete marker is kept until the major compaction 
+The amount of time to delay purging of delete markers with future timestamps. If
+      unset, or set to 0, all delete markers, including those with future timestamps, are purged
+      during the next major compaction. Otherwise, a delete marker is kept until the major compaction
       which occurs after the marker's timestamp plus the value of this setting, in milliseconds.
-    
+
 +
 .Default
 `0`
 
-  
+
 [[hbase.offpeak.start.hour]]
 *`hbase.offpeak.start.hour`*::
 +
@@ -1157,7 +1110,7 @@ The start of off-peak hours, expressed as an integer between 0 and 23, inclusive
 .Default
 `-1`
 
-  
+
 [[hbase.offpeak.end.hour]]
 *`hbase.offpeak.end.hour`*::
 +
@@ -1168,7 +1121,7 @@ The end of off-peak hours, expressed as an integer between 0 and 23, inclusive.
 .Default
 `-1`
 
-  
+
 [[hbase.regionserver.thread.compaction.throttle]]
 *`hbase.regionserver.thread.compaction.throttle`*::
 +
@@ -1184,19 +1137,19 @@ There are two different thread pools for compactions, one for large compactions
 .Default
 `2684354560`
 
-  
+
 [[hbase.hstore.compaction.kv.max]]
 *`hbase.hstore.compaction.kv.max`*::
 +
 .Description
 The maximum number of KeyValues to read and then write in a batch when flushing or
       compacting. Set this lower if you have big KeyValues and problems with Out Of Memory
-      Exceptions Set this higher if you have wide, small rows. 
+      Exceptions Set this higher if you have wide, small rows.
 +
 .Default
 `10`
 
-  
+
 [[hbase.storescanner.parallel.seek.enable]]
 *`hbase.storescanner.parallel.seek.enable`*::
 +
@@ -1208,7 +1161,7 @@ The maximum number of KeyValues to read and then write in a batch when flushing
 .Default
 `false`
 
-  
+
 [[hbase.storescanner.parallel.seek.threads]]
 *`hbase.storescanner.parallel.seek.threads`*::
 +
@@ -1219,7 +1172,7 @@ The maximum number of KeyValues to read and then write in a batch when flushing
 .Default
 `10`
 
-  
+
 [[hfile.block.cache.size]]
 *`hfile.block.cache.size`*::
 +
@@ -1232,7 +1185,7 @@ Percentage of maximum heap (-Xmx setting) to allocate to block cache
 .Default
 `0.4`
 
-  
+
 [[hfile.block.index.cacheonwrite]]
 *`hfile.block.index.cacheonwrite`*::
 +
@@ -1243,7 +1196,7 @@ This allows to put non-root multi-level index blocks into the block
 .Default
 `false`
 
-  
+
 [[hfile.index.block.max.size]]
 *`hfile.index.block.max.size`*::
 +
@@ -1255,31 +1208,33 @@ When the size of a leaf-level, intermediate-level, or root-level
 .Default
 `131072`
 
-  
+
 [[hbase.bucketcache.ioengine]]
 *`hbase.bucketcache.ioengine`*::
 +
 .Description
-Where to store the contents of the bucketcache. One of: onheap, 
-      offheap, or file. If a file, set it to file:PATH_TO_FILE. See https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html for more information.
-    
+Where to store the contents of the bucketcache. One of: onheap,
+      offheap, or file. If a file, set it to file:PATH_TO_FILE.
+      See https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/io/hfile/CacheConfig.html
+      for more information.
+
 +
 .Default
 ``
 
-  
+
 [[hbase.bucketcache.combinedcache.enabled]]
 *`hbase.bucketcache.combinedcache.enabled`*::
 +
 .Description
-Whether or not the bucketcache is used in league with the LRU 
-      on-heap block cache. In this mode, indices and blooms are kept in the LRU 
+Whether or not the bucketcache is used in league with the LRU
+      on-heap block cache. In this mode, indices and blooms are kept in the LRU
       blockcache and the data blocks are kept in the bucketcache.
 +
 .Default
 `true`
 
-  
+
 [[hbase.bucketcache.size]]
 *`hbase.bucketcache.size`*::
 +
@@ -1290,19 +1245,19 @@ Used along with bucket cache, this is a float that EITHER represents a percentag
 .Default
 `0` when specified as a float
 
-  
-[[hbase.bucketcache.sizes]]
-*`hbase.bucketcache.sizes`*::
+
+[[hbase.bucketcache.bucket.sizes]]
+*`hbase.bucketcache.bucket.sizes`*::
 +
 .Description
-A comma-separated list of sizes for buckets for the bucketcache 
-      if you use multiple sizes. Should be a list of block sizes in order from smallest 
+A comma-separated list of sizes for buckets for the bucketcache
+      if you use multiple sizes. Should be a list of block sizes in order from smallest
       to largest. The sizes you use will depend on your data access patterns.
 +
 .Default
 ``
 
-  
+
 [[hfile.format.version]]
 *`hfile.format.version`*::
 +
@@ -1310,13 +1265,13 @@ A comma-separated list of sizes for buckets for the bucketcache
 The HFile format version to use for new files.
       Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags).
       Distributed Log Replay requires that tags are enabled. Also see the configuration
-      'hbase.replication.rpc.codec'. 
-      
+      'hbase.replication.rpc.codec'.
+
 +
 .Default
 `3`
 
-  
+
 [[hfile.block.bloom.cacheonwrite]]
 *`hfile.block.bloom.cacheonwrite`*::
 +
@@ -1326,7 +1281,7 @@ Enables cache-on-write for inline blocks of a compound Bloom filter.
 .Default
 `false`
 
-  
+
 [[io.storefile.bloom.block.size]]
 *`io.storefile.bloom.block.size`*::
 +
@@ -1339,7 +1294,7 @@ The size in bytes of a single block ("chunk") of a compound Bloom
 .Default
 `131072`
 
-  
+
 [[hbase.rs.cacheblocksonwrite]]
 *`hbase.rs.cacheblocksonwrite`*::
 +
@@ -1350,7 +1305,7 @@ Whether an HFile block should be added to the block cache when the
 .Default
 `false`
 
-  
+
 [[hbase.rpc.timeout]]
 *`hbase.rpc.timeout`*::
 +
@@ -1362,7 +1317,7 @@ This is for the RPC layer to define how long HBase client applications
 .Default
 `60000`
 
-  
+
 [[hbase.rpc.shortoperation.timeout]]
 *`hbase.rpc.shortoperation.timeout`*::
 +
@@ -1375,7 +1330,7 @@ This is another version of "hbase.rpc.timeout". For those RPC operation
 .Default
 `10000`
 
-  
+
 [[hbase.ipc.client.tcpnodelay]]
 *`hbase.ipc.client.tcpnodelay`*::
 +
@@ -1386,7 +1341,7 @@ Set no delay on rpc socket connections.  See
 .Default
 `true`
 
-  
+
 [[hbase.master.keytab.file]]
 *`hbase.master.keytab.file`*::
 +
@@ -1397,7 +1352,7 @@ Full path to the kerberos keytab file to use for logging in
 .Default
 ``
 
-  
+
 [[hbase.master.kerberos.principal]]
 *`hbase.master.kerberos.principal`*::
 +
@@ -1411,7 +1366,7 @@ Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
 .Default
 ``
 
-  
+
 [[hbase.regionserver.keytab.file]]
 *`hbase.regionserver.keytab.file`*::
 +
@@ -1422,7 +1377,7 @@ Full path to the kerberos keytab file to use for logging in
 .Default
 ``
 
-  
+
 [[hbase.regionserver.kerberos.principal]]
 *`hbase.regionserver.kerberos.principal`*::
 +
@@ -1437,7 +1392,7 @@ Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
 .Default
 ``
 
-  
+
 [[hadoop.policy.file]]
 *`hadoop.policy.file`*::
 +
@@ -1449,7 +1404,7 @@ The policy configuration file used by RPC servers to make
 .Default
 `hbase-policy.xml`
 
-  
+
 [[hbase.superuser]]
 *`hbase.superuser`*::
 +
@@ -1461,7 +1416,7 @@ List of users or groups (comma-separated), who are allowed
 .Default
 ``
 
-  
+
 [[hbase.auth.key.update.interval]]
 *`hbase.auth.key.update.interval`*::
 +
@@ -1472,7 +1427,7 @@ The update interval for master key for authentication tokens
 .Default
 `86400000`
 
-  
+
 [[hbase.auth.token.max.lifetime]]
 *`hbase.auth.token.max.lifetime`*::
 +
@@ -1483,7 +1438,7 @@ The maximum lifetime in milliseconds after which an
 .Default
 `604800000`
 
-  
+
 [[hbase.ipc.client.fallback-to-simple-auth-allowed]]
 *`hbase.ipc.client.fallback-to-simple-auth-allowed`*::
 +
@@ -1498,7 +1453,7 @@ When a client is configured to attempt a secure connection, but attempts to
 .Default
 `false`
 
-  
+
 [[hbase.display.keys]]
 *`hbase.display.keys`*::
 +
@@ -1510,7 +1465,7 @@ When this is set to true the webUI and such will display all start/end keys
 .Default
 `true`
 
-  
+
 [[hbase.coprocessor.region.classes]]
 *`hbase.coprocessor.region.classes`*::
 +
@@ -1524,7 +1479,7 @@ A comma-separated list of Coprocessors that are loaded by
 .Default
 ``
 
-  
+
 [[hbase.rest.port]]
 *`hbase.rest.port`*::
 +
@@ -1534,7 +1489,7 @@ The port for the HBase REST server.
 .Default
 `8080`
 
-  
+
 [[hbase.rest.readonly]]
 *`hbase.rest.readonly`*::
 +
@@ -1546,7 +1501,7 @@ Defines the mode the REST server will be started in. Possible values are:
 .Default
 `false`
 
-  
+
 [[hbase.rest.threads.max]]
 *`hbase.rest.threads.max`*::
 +
@@ -1561,7 +1516,7 @@ The maximum number of threads of the REST server thread pool.
 .Default
 `100`
 
-  
+
 [[hbase.rest.threads.min]]
 *`hbase.rest.threads.min`*::
 +
@@ -1573,7 +1528,7 @@ The minimum number of threads of the REST server thread pool.
 .Default
 `2`
 
-  
+
 [[hbase.rest.support.proxyuser]]
 *`hbase.rest.support.proxyuser`*::
 +
@@ -1583,7 +1538,7 @@ Enables running the REST server to support proxy-user mode.
 .Default
 `false`
 
-  
+
 [[hbase.defaults.for.version.skip]]
 *`hbase.defaults.for.version.skip`*::
 +
@@ -1592,14 +1547,14 @@ Set to true to skip the 'hbase.defaults.for.version' check.
     Setting this to true can be useful in contexts other than
     the other side of a maven generation; i.e. running in an
     ide.  You'll want to set this boolean to true to avoid
-    seeing the RuntimException complaint: "hbase-default.xml file
+    seeing the RuntimeException complaint: "hbase-default.xml file
     seems to be for and old version of HBase (\${hbase.version}), this
     version is X.X.X-SNAPSHOT"
 +
 .Default
 `false`
 
-  
+
 [[hbase.coprocessor.master.classes]]
 *`hbase.coprocessor.master.classes`*::
 +
@@ -1614,7 +1569,7 @@ A comma-separated list of
 .Default
 ``
 
-  
+
 [[hbase.coprocessor.abortonerror]]
 *`hbase.coprocessor.abortonerror`*::
 +
@@ -1629,17 +1584,7 @@ Set to true to cause the hosting server (master or regionserver)
 .Default
 `true`
 
-  
-[[hbase.online.schema.update.enable]]
-*`hbase.online.schema.update.enable`*::
-+
-.Description
-Set true to enable online schema changes.
-+
-.Default
-`true`
 
-  
 [[hbase.table.lock.enable]]
 *`hbase.table.lock.enable`*::
 +
@@ -1651,7 +1596,7 @@ Set to true to enable locking the table in zookeeper for schema change operation
 .Default
 `true`
 
-  
+
 [[hbase.table.max.rowsize]]
 *`hbase.table.max.rowsize`*::
 +
@@ -1660,12 +1605,12 @@ Set to true to enable locking the table in zookeeper for schema change operation
       Maximum size of single row in bytes (default is 1 Gb) for Get'ting
       or Scan'ning without in-row scan flag set. If row size exceeds this limit
       RowTooBigException is thrown to client.
-    
+
 +
 .Default
 `1073741824`
 
-  
+
 [[hbase.thrift.minWorkerThreads]]
 *`hbase.thrift.minWorkerThreads`*::
 +
@@ -1676,7 +1621,7 @@ The "core size" of the thread pool. New threads are created on every
 .Default
 `16`
 
-  
+
 [[hbase.thrift.maxWorkerThreads]]
 *`hbase.thrift.maxWorkerThreads`*::
 +
@@ -1688,7 +1633,7 @@ The maximum size of the thread pool. When the pending request queue
 .Default
 `1000`
 
-  
+
 [[hbase.thrift.maxQueuedRequests]]
 *`hbase.thrift.maxQueuedRequests`*::
 +
@@ -1701,21 +1646,7 @@ The maximum number of pending Thrift connections waiting in the queue. If
 .Default
 `1000`
 
-  
-[[hbase.thrift.htablepool.size.max]]
-*`hbase.thrift.htablepool.size.max`*::
-+
-.Description
-The upper bound for the table pool used in the Thrift gateways server.
-      Since this is per table name, we assume a single table and so with 1000 default
-      worker threads max this is set to a matching number. For other workloads this number
-      can be adjusted as needed.
-    
-+
-.Default
-`1000`
 
-  
 [[hbase.regionserver.thrift.framed]]
 *`hbase.regionserver.thrift.framed`*::
 +
@@ -1724,12 +1655,12 @@ Use Thrift TFramedTransport on the server side.
       This is the recommended transport for thrift servers and requires a similar setting
       on the client side. Changing this to false will select the default transport,
       vulnerable to DoS when malformed requests are issued due to THRIFT-601.
-    
+
 +
 .Default
 `false`
 
-  
+
 [[hbase.regionserver.thrift.framed.max_frame_size_in_mb]]
 *`hbase.regionserver.thrift.framed.max_frame_size_in_mb`*::
 +
@@ -1739,7 +1670,7 @@ Default frame size when using framed transport
 .Default
 `2`
 
-  
+
 [[hbase.regionserver.thrift.compact]]
 *`hbase.regionserver.thrift.compact`*::
 +
@@ -1749,7 +1680,7 @@ Use Thrift TCompactProtocol binary serialization protocol.
 .Default
 `false`
 
-  
+
 [[hbase.data.umask.enable]]
 *`hbase.data.umask.enable`*::
 +
@@ -1760,7 +1691,7 @@ Enable, if true, that file permissions should be assigned
 .Default
 `false`
 
-  
+
 [[hbase.data.umask]]
 *`hbase.data.umask`*::
 +
@@ -1771,32 +1702,7 @@ File permissions that should be used to write data
 .Default
 `000`
 
-  
-[[hbase.metrics.showTableName]]
-*`hbase.metrics.showTableName`*::
-+
-.Description
-Whether to include the prefix "tbl.tablename" in per-column family metrics.
-	If true, for each metric M, per-cf metrics will be reported for tbl.T.cf.CF.M, if false,
-	per-cf metrics will be aggregated by column-family across tables, and reported for cf.CF.M.
-	In both cases, the aggregated metric M across tables and cfs will be reported.
-+
-.Default
-`true`
-
-  
-[[hbase.metrics.exposeOperationTimes]]
-*`hbase.metrics.exposeOperationTimes`*::
-+
-.Description
-Whether to report metrics about time taken performing an
-      operation on the region server.  Get, Put, Delete, Increment, and Append can all
-      have their times exposed through Hadoop metrics per CF and per region.
-+
-.Default
-`true`
 
-  
 [[hbase.snapshot.enabled]]
 *`hbase.snapshot.enabled`*::
 +
@@ -1806,7 +1712,7 @@ Set to true to allow snapshots to be taken / restored / cloned.
 .Default
 `true`
 
-  
+
 [[hbase.snapshot.restore.take.failsafe.snapshot]]
 *`hbase.snapshot.restore.take.failsafe.snapshot`*::
 +
@@ -1818,7 +1724,7 @@ Set to true to take a snapshot before the restore operation.
 .Default
 `true`
 
-  
+
 [[hbase.snapshot.restore.failsafe.name]]
 *`hbase.snapshot.restore.failsafe.name`*::
 +
@@ -1830,7 +1736,7 @@ Name of the failsafe snapshot taken by the restore operation.
 .Default
 `hbase-failsafe-{snapshot.name}-{restore.timestamp}`
 
-  
+
 [[hbase.server.compactchecker.interval.multiplier]]
 *`hbase.server.compactchecker.interval.multiplier`*::
 +
@@ -1845,7 +1751,7 @@ The number that determines how often we scan to see if compaction is necessary.
 .Default
 `1000`
 
-  
+
 [[hbase.lease.recovery.timeout]]
 *`hbase.lease.recovery.timeout`*::
 +
@@ -1855,7 +1761,7 @@ How long we wait on dfs lease recovery in total before giving up.
 .Default
 `900000`
 
-  
+
 [[hbase.lease.recovery.dfs.timeout]]
 *`hbase.lease.recovery.dfs.timeout`*::
 +
@@ -1869,7 +1775,7 @@ How long between dfs recover lease invocations. Should be larger than the sum of
 .Default
 `64000`
 
-  
+
 [[hbase.column.max.version]]
 *`hbase.column.max.version`*::
 +
@@ -1880,7 +1786,7 @@ New column family descriptors will use this value as the default number of versi
 .Default
 `1`
 
-  
+
 [[hbase.dfs.client.read.shortcircuit.buffer.size]]
 *`hbase.dfs.client.read.shortcircuit.buffer.size`*::
 +
@@ -1894,12 +1800,12 @@ If the DFSClient configuration
     direct memory.  So, we set it down from the default.  Make
     it > the default hbase block size set in the HColumnDescriptor
     which is usually 64k.
-    
+
 +
 .Default
 `131072`
 
-  
+
 [[hbase.regionserver.checksum.verify]]
 *`hbase.regionserver.checksum.verify`*::
 +
@@ -1914,13 +1820,13 @@ If the DFSClient configuration
         fails, we will switch back to using HDFS checksums (so do not disable HDFS
         checksums!  And besides this feature applies to hfiles only, not to WALs).
         If this parameter is set to false, then hbase will not verify any checksums,
-        instead it will depend on checksum verification being done in the HDFS client.  
-    
+        instead it will depend on checksum verification being done in the HDFS client.
+
 +
 .Default
 `true`
 
-  
+
 [[hbase.hstore.bytes.per.checksum]]
 *`hbase.hstore.bytes.per.checksum`*::
 +
@@ -1928,12 +1834,12 @@ If the DFSClient configuration
 
         Number of bytes in a newly created checksum chunk for HBase-level
         checksums in hfile blocks.
-    
+
 +
 .Default
 `16384`
 
-  
+
 [[hbase.hstore.checksum.algorithm]]
 *`hbase.hstore.checksum.algorithm`*::
 +
@@ -1941,12 +1847,12 @@ If the DFSClient configuration
 
       Name of an algorithm that is used to compute checksums. Possible values
       are NULL, CRC32, CRC32C.
-    
+
 +
 .Default
 `CRC32`
 
-  
+
 [[hbase.status.published]]
 *`hbase.status.published`*::
 +
@@ -1956,60 +1862,60 @@ If the DFSClient configuration
       When a region server dies and its recovery starts, the master will push this information
       to the client application, to let them cut the connection immediately instead of waiting
       for a timeout.
-    
+
 +
 .Default
 `false`
 
-  
+
 [[hbase.status.publisher.class]]
 *`hbase.status.publisher.class`*::
 +
 .Description
 
       Implementation of the status publication with a multicast message.
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.master.ClusterStatusPublisher$MulticastPublisher`
 
-  
+
 [[hbase.status.listener.class]]
 *`hbase.status.listener.class`*::
 +
 .Description
 
       Implementation of the status listener with a multicast message.
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener`
 
-  
+
 [[hbase.status.multicast.address.ip]]
 *`hbase.status.multicast.address.ip`*::
 +
 .Description
 
       Multicast address to use for the status publication by multicast.
-    
+
 +
 .Default
 `226.1.1.3`
 
-  
+
 [[hbase.status.multicast.address.port]]
 *`hbase.status.multicast.address.port`*::
 +
 .Description
 
       Multicast port to use for the status publication by multicast.
-    
+
 +
 .Default
 `16100`
 
-  
+
 [[hbase.dynamic.jars.dir]]
 *`hbase.dynamic.jars.dir`*::
 +
@@ -2019,12 +1925,12 @@ If the DFSClient configuration
       dynamically by the region server without the need to restart. However,
       an already loaded filter/co-processor class would not be un-loaded. See
       HBASE-1936 for more details.
-    
+
 +
 .Default
 `${hbase.rootdir}/lib`
 
-  
+
 [[hbase.security.authentication]]
 *`hbase.security.authentication`*::
 +
@@ -2032,24 +1938,24 @@ If the DFSClient configuration
 
       Controls whether or not secure authentication is enabled for HBase.
       Possible values are 'simple' (no authentication), and 'kerberos'.
-    
+
 +
 .Default
 `simple`
 
-  
+
 [[hbase.rest.filter.classes]]
 *`hbase.rest.filter.classes`*::
 +
 .Description
 
       Servlet filters for REST service.
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.rest.filter.GzipFilter`
 
-  
+
 [[hbase.master.loadbalancer.class]]
 *`hbase.master.loadbalancer.class`*::
 +
@@ -2060,12 +1966,12 @@ If the DFSClient configuration
       http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
       It replaces the DefaultLoadBalancer as the default (since renamed
       as the SimpleLoadBalancer).
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer`
 
-  
+
 [[hbase.security.exec.permission.checks]]
 *`hbase.security.exec.permission.checks`*::
 +
@@ -2081,28 +1987,28 @@ If the DFSClient configuration
       section of the HBase online manual. For more information on granting or
       revoking permissions using the AccessController, see the security
       section of the HBase online manual.
-    
+
 +
 .Default
 `false`
 
-  
+
 [[hbase.procedure.regionserver.classes]]
 *`hbase.procedure.regionserver.classes`*::
 +
 .Description
-A comma-separated list of 
-    org.apache.hadoop.hbase.procedure.RegionServerProcedureManager procedure managers that are 
-    loaded by default on the active HRegionServer process. The lifecycle methods (init/start/stop) 
-    will be called by the active HRegionServer process to perform the specific globally barriered 
-    procedure. After implementing your own RegionServerProcedureManager, just put it in 
+A comma-separated list of
+    org.apache.hadoop.hbase.procedure.RegionServerProcedureManager procedure managers that are
+    loaded by default on the active HRegionServer process. The lifecycle methods (init/start/stop)
+    will be called by the active HRegionServer process to perform the specific globally barriered
+    procedure. After implementing your own RegionServerProcedureManager, just put it in
     HBase's classpath and add the fully qualified class name here.
-    
+
 +
 .Default
 ``
 
-  
+
 [[hbase.procedure.master.classes]]
 *`hbase.procedure.master.classes`*::
 +
@@ -2117,7 +2023,7 @@ A comma-separated list of
 .Default
 ``
 
-  
+
 [[hbase.coordinated.state.manager.class]]
 *`hbase.coordinated.state.manager.class`*::
 +
@@ -2127,7 +2033,7 @@ Fully qualified name of class implementing coordinated state manager.
 .Default
 `org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager`
 
-  
+
 [[hbase.regionserver.storefile.refresh.period]]
 *`hbase.regionserver.storefile.refresh.period`*::
 +
@@ -2140,12 +2046,12 @@ Fully qualified name of class implementing coordinated state manager.
       extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL
       (hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger
       value is also recommended with this setting.
-    
+
 +
 .Default
 `0`
 
-  
+
 [[hbase.region.replica.replication.enabled]]
 *`hbase.region.replica.replication.enabled`*::
 +
@@ -2153,36 +2059,35 @@ Fully qualified name of class implementing coordinated state manager.
 
       Whether asynchronous WAL replication to the secondary region replicas is enabled or not.
       If this is enabled, a replication peer named "region_replica_replication" will be created
-      which will tail the logs and replicate the mutatations to region replicas for tables that
+      which will tail the logs and replicate the mutations to region replicas for tables that
       have region replication > 1. If this is enabled once, disabling this replication also
       requires disabling the replication peer using shell or ReplicationAdmin java class.
-      Replication to secondary region replicas works over standard inter-cluster replication. 
-      So replication, if disabled explicitly, also has to be enabled by setting "hbase.replication" 
-      to true for this feature to work.
-    
+      Replication to secondary region replicas works over standard inter-cluster replication.
+
+
 +
 .Default
 `false`
 
-  
+
 [[hbase.http.filter.initializers]]
 *`hbase.http.filter.initializers`*::
 +
 .Description
 
-      A comma separated list of class names. Each class in the list must extend 
-      org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will 
-      be initialized. Then, the Filter will be applied to all user facing jsp 
-      and servlet web pages. 
+      A comma separated list of class names. Each class in the list must extend
+      org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will
+      be initialized. Then, the Filter will be applied to all user facing jsp
+      and servlet web pages.
       The ordering of the list defines the ordering of the filters.
-      The default StaticUserWebFilter add a user principal as defined by the 
+      The default StaticUserWebFilter add a user principal as defined by the
       hbase.http.staticuser.user property.
-    
+
 +
 .Default
 `org.apache.hadoop.hbase.http.lib.StaticUserWebFilter`
 
-  
+
 [[hbase.security.visibility.mutations.checkauths]]
 *`hbase.security.visibility.mutations.checkauths`*::
 +
@@ -2190,41 +2095,41 @@ Fully qualified name of class implementing coordinated state manager.
 
       This property if enabled, will check whether the labels in the visibility expression are associated
       with the user issuing the mutation
-    
+
 +
 .Default
 `false`
 
-  
+
 [[hbase.http.max.threads]]
 *`hbase.http.max.threads`*::
 +
 .Description
 
-      The maximum number of threads that the HTTP Server will create in its 
+      The maximum number of threads that the HTTP Server will create in its
       ThreadPool.
-    
+
 +
 .Default
 `10`
 
-  
+
 [[hbase.replication.rpc.codec]]
 *`hbase.replication.rpc.codec`*::
 +
 .Description
 
   		The codec that is to be used when replication is enabled so that
-  		the tags are also replicated. This is used along with HFileV3 which 
+  		the tags are also replicated. This is used along with HFileV3 which
   		supports tags in them.  If tags are not used or if the hfile version used
   		is HFileV2 then KeyValueCodec can be used as the replication codec. Note that
   		using KeyValueCodecWithTags for replication when there are no tags causes no harm.
-  	
+
 +
 .Default
 `org.apache.hadoop.hbase.codec.KeyValueCodecWithTags`
 
-  
+
 [[hbase.http.staticuser.user]]
 *`hbase.http.staticuser.user`*::
 +
@@ -2233,12 +2138,12 @@ Fully qualified name of class implementing coordinated state manager.
       The user name to filter as, on static web filters
       while rendering content. An example use is the HDFS
       web UI (user to be used for browsing files).
-    
+
 +
 .Default
 `dr.stack`
 
-  
+
 [[hbase.regionserver.handler.abort.on.error.percent]]
 *`hbase.regionserver.handler.abort.on.error.percent`*::
 +
@@ -2251,4 +2156,3 @@ The percent of region server RPC threads failed to abort RS.
 .Default
 `0.5`
 
-  
\ No newline at end of file


[2/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/protobuf.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/protobuf.adoc b/src/main/asciidoc/_chapters/protobuf.adoc
new file mode 100644
index 0000000..fa63127
--- /dev/null
+++ b/src/main/asciidoc/_chapters/protobuf.adoc
@@ -0,0 +1,153 @@
+////
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+////
+
+[[protobuf]]
+= Protobuf in HBase
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+HBase uses Google's link:http://protobuf.protobufs[protobufs] wherever
+it persists metadata -- in the tail of hfiles or Cells written by
+HBase into the system hbase;meta table or when HBase writes znodes
+to zookeeper, etc. -- and when it passes objects over the wire making
+xref:hbase.rpc[RPCs]. HBase uses protobufs to describe the RPC
+Interfaces (Services) we expose to clients, for example the `Admin` and `Client`
+Interfaces that the RegionServer fields,
+or specifying the arbitrary extensions added by developers via our
+xref:cp[Coprocessor Endpoint] mechanism.
+In this chapter we go into detail for  developers who are looking to
+understand better how it all works. This chapter is of particular
+use to those who would amend or extend HBase functionality.
+
+== Protobuf
+
+With protobuf, you describe serializations and services in a `.protos` file.
+You then feed these descriptors to a protobuf tool, the `protoc` binary,
+to generate classes that can marshall and unmarshall the described serializations
+and field the specified Services.
+
+See the `README.txt` in the HBase sub-modules for detail on how
+to run the class generation on a per-module basis;
+e.g. see `hbase-protocol/README.txt` for how to generated protobuf classes
+in the hbase-protocol module.
+
+In HBase, `.proto` files are either in the `hbase-protocol` module, a module
+dedicated to hosting the common proto files and the protoc generated classes
+that HBase uses internally serializing metadata or, for extensions to hbase
+such as REST or Coprocessor Endpoints that need their own descriptors, their
+protos are located inside the function's hosting module: e.g. `hbase-rest`
+is home to the REST proto files and the `hbase-rsgroup` table grouping
+Coprocessor Endpoint has all protos that have to do with table grouping.
+
+Protos are hosted by the module that makes use of them. While
+this makes it so generation of protobuf classes is distributed, done
+per module, we do it this way so modules encapsulate all to do with
+the functionality they bring to hbase.
+
+Extensions whether REST or Coprocessor Endpoints will make use
+of core HBase protos found back in the hbase-protocol module. They'll
+use these core protos when they want to serialize a Cell or a Put or
+refer to a particular node via ServerName, etc., as part of providing the
+CPEP Service. Going forward, after the release of hbase-2.0.0, this
+practice needs to whither. We'll make plain why in the later
+xref:shaded.protobuf[hbase-2.0.0] section.
+
+[[shaded.protobuf]]
+=== hbase-2.0.0 and the shading of protobufs (HBASE-15638)
+
+As of hbase-2.0.0, our protobuf usage gets a little more involved. HBase
+core protobuf references are offset so as to refer to a private,
+bundled protobuf. Core stops referring to protobuf
+classes at com.google.protobuf.* and instead references protobuf at
+the HBase-specific offset
+org.apache.hadoop.hbase.shaded.com.google.protobuf.*.  We do this indirection
+so hbase core can evolve its protobuf version independent of whatever our
+dependencies rely on. For instance, HDFS serializes using protobuf.
+HDFS is on our CLASSPATH. Without the above described indirection, our
+protobuf versions would have to align. HBase would be stuck
+on the HDFS protobuf version until HDFS decided upgrade. HBase
+and HDFS verions would be tied.
+
+We had to move on from protobuf-2.5.0 because we need facilities
+added in protobuf-3.1.0; in particular being able to save on
+copies and avoiding bringing protobufs onheap for
+serialization/deserialization.
+
+In hbase-2.0.0, we introduced a new module, `hbase-protocol-shaded`
+inside which we contained all to do with protobuf and its subsequent
+relocation/shading. This module is in essence a copy of much of the old
+`hbase-protocol` but with an extra shading/relocation step (see the `README.txt`
+and the `poms.xml` in this module for more on how to trigger this
+effect and how it all works). Core was moved to depend on this new
+module.
+
+That said, a complication arises around Coprocessor Endpoints (CPEPs).
+CPEPs depend on public HBase APIs that reference protobuf classes at
+`com.google.protobuf.*` explicitly. For example, in our Table Interface
+we have the below as the means by which you obtain a CPEP Service
+to make invocations against:
+
+[source,java]
+----
+...
+  <T extends com.google.protobuf.Service,R> Map<byte[],R> coprocessorService(
+   Class<T> service, byte[] startKey, byte[] endKey,
+     org.apache.hadoop.hbase.client.coprocessor.Batch.Call<T,R> callable)
+  throws com.google.protobuf.ServiceException, Throwable
+----
+
+Existing CPEPs will have made reference to core HBase protobufs
+specifying ServerNames or carrying Mutations.
+So as to continue being able to service CPEPs and their references
+to `com.google.protobuf.*` across the upgrade to hbase-2.0.0 and beyond,
+HBase needs to be able to deal with both
+`com.google.protobuf.*` references and its internal offset
+`org.apache.hadoop.hbase.shaded.com.google.protobuf.*` protobufs.
+
+The `hbase-protocol-shaded` module hosts all
+protobufs used by HBase core as well as the internal shaded version of
+protobufs that hbase depends on. hbase-client and hbase-server, etc.,
+depend on this module.
+
+But for the vestigial CPEP references to the (non-shaded) content of
+`hbase-protocol`, we keep around most of this  module going forward
+just so it is available to CPEPs.  Retaining the most of `hbase-protocol`
+makes for overlapping, 'duplicated' proto instances where some exist as
+non-shaded/non-relocated here in their old module
+location but also in the new location, shaded under
+`hbase-protocol-shaded`. In other words, there is an instance
+of the generated protobuf class
+`org.apache.hadoop.hbase.protobuf.generated.ServerName`
+in hbase-protocol and another generated instance that is the same in all
+regards except its protobuf references are to the internal shaded
+version at `org.apache.hadoop.hbase.shaded.protobuf.generated.ServerName`
+(note the 'shaded' addition in the middle of the package name).
+
+If you extend a proto in `hbase-protocol-shaded` for  internal use,
+consider extending it also in
+`hbase-protocol` (and regenerating).
+
+Going forward, we will provide a new module of common types for use
+by CPEPs that will have the same guarantees against change as does our
+public API. TODO.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/rpc.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/rpc.adoc b/src/main/asciidoc/_chapters/rpc.adoc
index 43e7156..1d363eb 100644
--- a/src/main/asciidoc/_chapters/rpc.adoc
+++ b/src/main/asciidoc/_chapters/rpc.adoc
@@ -47,7 +47,7 @@ For more background on how we arrived at this spec., see link:https://docs.googl
 
 
 . A wire-format we can evolve
-. A format that does not require our rewriting server core or radically changing its current architecture (for later).        
+. A format that does not require our rewriting server core or radically changing its current architecture (for later).
 
 === TODO
 
@@ -58,7 +58,7 @@ For more background on how we arrived at this spec., see link:https://docs.googl
 . Diagram on how it works
 . A grammar that succinctly describes the wire-format.
   Currently we have these words and the content of the rpc protobuf idl but a grammar for the back and forth would help with groking rpc.
-  Also, a little state machine on client/server interactions would help with understanding (and ensuring correct implementation).        
+  Also, a little state machine on client/server interactions would help with understanding (and ensuring correct implementation).
 
 === RPC
 
@@ -71,14 +71,15 @@ Optionally, Cells(KeyValues) can be passed outside of protobufs in follow-behind
 
 
 
-For more detail on the protobufs involved, see the link:http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/protobuf/RPC.proto?view=markup[RPC.proto]            file in trunk.
+For more detail on the protobufs involved, see the
+link:https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=blob;f=hbase-protocol/src/main/protobuf/RPC.proto;hb=HEAD[RPC.proto]            file in master.
 
 ==== Connection Setup
 
 Client initiates connection.
 
 ===== Client
-On connection setup, client sends a preamble followed by a connection header. 
+On connection setup, client sends a preamble followed by a connection header.
 
 .<preamble>
 [source]
@@ -105,7 +106,7 @@ After client sends preamble and connection header, server does NOT respond if su
 No response means server is READY to accept requests and to give out response.
 If the version or authentication in the preamble is not agreeable or the server has trouble parsing the preamble, it will throw a org.apache.hadoop.hbase.ipc.FatalConnectionException explaining the error and will then disconnect.
 If the client in the connection header -- i.e.
-the protobuf'd Message that comes after the connection preamble -- asks for for a Service the server does not support or a codec the server does not have, again we throw a FatalConnectionException with explanation.
+the protobuf'd Message that comes after the connection preamble -- asks for a Service the server does not support or a codec the server does not have, again we throw a FatalConnectionException with explanation.
 
 ==== Request
 
@@ -117,7 +118,7 @@ The header includes the method name and optionally, metadata on the optional Cel
 The parameter type suits the method being invoked: i.e.
 if we are doing a getRegionInfo request, the protobuf Message param will be an instance of GetRegionInfoRequest.
 The response will be a GetRegionInfoResponse.
-The CellBlock is optionally used ferrying the bulk of the RPC data: i.e Cells/KeyValues.
+The CellBlock is optionally used ferrying the bulk of the RPC data: i.e. Cells/KeyValues.
 
 ===== Request Parts
 
@@ -181,7 +182,7 @@ Codecs will live on the server for all time so old clients can connect.
 
 .Constraints
 In some part, current wire-format -- i.e.
-all requests and responses preceeded by a length -- has been dictated by current server non-async architecture.
+all requests and responses preceded by a length -- has been dictated by current server non-async architecture.
 
 .One fat pb request or header+param
 We went with pb header followed by pb param making a request and a pb header followed by pb response for now.
@@ -190,7 +191,7 @@ Doing header+param rather than a single protobuf Message with both header and pa
 . Is closer to what we currently have
 . Having a single fat pb requires extra copying putting the already pb'd param into the body of the fat request pb (and same making result)
 . We can decide whether to accept the request or not before we read the param; for example, the request might be low priority.
-  As is, we read header+param in one go as server is currently implemented so this is a TODO.            
+  As is, we read header+param in one go as server is currently implemented so this is a TODO.
 
 The advantages are minor.
 If later, fat request has clear advantage, can roll out a v2 later.
@@ -204,18 +205,18 @@ Codec must implement hbase's `Codec` Interface.
 After connection setup, all passed cellblocks will be sent with this codec.
 The server will return cellblocks using this same codec as long as the codec is on the servers' CLASSPATH (else you will get `UnsupportedCellCodecException`).
 
-To change the default codec, set `hbase.client.default.rpc.codec`. 
+To change the default codec, set `hbase.client.default.rpc.codec`.
 
 To disable cellblocks completely and to go pure protobuf, set the default to the empty String and do not specify a codec in your Configuration.
 So, set `hbase.client.default.rpc.codec` to the empty string and do not set `hbase.client.rpc.codec`.
 This will cause the client to connect to the server with no codec specified.
 If a server sees no codec, it will return all responses in pure protobuf.
-Running pure protobuf all the time will be slower than running with cellblocks. 
+Running pure protobuf all the time will be slower than running with cellblocks.
 
 .Compression
-Uses hadoops compression codecs.
+Uses hadoop's compression codecs.
 To enable compressing of passed CellBlocks, set `hbase.client.rpc.compressor` to the name of the Compressor to use.
-Compressor must implement Hadoops' CompressionCodec Interface.
+Compressor must implement Hadoop's CompressionCodec Interface.
 After connection setup, all passed cellblocks will be sent compressed.
 The server will return cellblocks compressed using this same compressor as long as the compressor is on its CLASSPATH (else you will get `UnsupportedCompressionCodecException`).
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/schema_design.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc
index 9319c65..7b85d15 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -27,7 +27,19 @@
 :icons: font
 :experimental:
 
-A good general introduction on the strength and weaknesses modelling on the various non-rdbms datastores is Ian Varley's Master thesis, link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation: The Mixed Blessings of Non-Relational Databases]. Also, read <<keyvalue,keyvalue>> for how HBase stores data internally, and the section on <<schema.casestudies,schema.casestudies>>.
+A good introduction on the strength and weaknesses modelling on the various non-rdbms datastores is
+to be found in Ian Varley's Master thesis,
+link:http://ianvarley.com/UT/MR/Varley_MastersReport_Full_2009-08-07.pdf[No Relation: The Mixed Blessings of Non-Relational Databases].
+It is a little dated now but a good background read if you have a moment on how HBase schema modeling
+differs from how it is done in an RDBMS. Also,
+read <<keyvalue,keyvalue>> for how HBase stores data internally, and the section on <<schema.casestudies,schema.casestudies>>.
+
+The documentation on the Cloud Bigtable website, link:https://cloud.google.com/bigtable/docs/schema-design[Designing Your Schema],
+is pertinent and nicely done and lessons learned there equally apply here in HBase land; just divide
+any quoted values by ~10 to get what works for HBase: e.g. where it says individual values can be ~10MBs in size, HBase can do similar -- perhaps best
+to go smaller if you can -- and where it says a maximum of 100 column families in Cloud Bigtable, think ~10 when
+modeling on HBase.
+
 
 [[schema.creation]]
 ==  Schema Creation
@@ -41,7 +53,7 @@ Tables must be disabled when making ColumnFamily modifications, for example:
 
 Configuration config = HBaseConfiguration.create();
 Admin admin = new Admin(conf);
-String table = "myTable";
+TableName table = TableName.valueOf("myTable");
 
 admin.disableTable(table);
 
@@ -64,6 +76,50 @@ When changes are made to either Tables or ColumnFamilies (e.g. region size, bloc
 
 See <<store,store>> for more information on StoreFiles.
 
+[[table_schema_rules_of_thumb]]
+== Table Schema Rules Of Thumb
+
+There are many different data sets, with different access patterns and service-level
+expectations. Therefore, these rules of thumb are only an overview. Read the rest
+of this chapter to get more details after you have gone through this list.
+
+* Aim to have regions sized between 10 and 50 GB.
+* Aim to have cells no larger than 10 MB, or 50 MB if you use <<hbase_mob,mob>>. Otherwise,
+consider storing your cell data in HDFS and store a pointer to the data in HBase.
+* A typical schema has between 1 and 3 column families per table. HBase tables should
+not be designed to mimic RDBMS tables.
+* Around 50-100 regions is a good number for a table with 1 or 2 column families.
+Remember that a region is a contiguous segment of a column family.
+* Keep your column family names as short as possible. The column family names are
+stored for every value (ignoring prefix encoding). They should not be self-documenting
+and descriptive like in a typical RDBMS.
+* If you are storing time-based machine data or logging information, and the row key
+is based on device ID or service ID plus time, you can end up with a pattern where
+older data regions never have additional writes beyond a certain age. In this type
+of situation, you end up with a small number of active regions and a large number
+of older regions which have no new writes. For these situations, you can tolerate
+a larger number of regions because your resource consumption is driven by the active
+regions only.
+* If only one column family is busy with writes, only that column family accomulates
+memory. Be aware of write patterns when allocating resources.
+
+[[regionserver_sizing_rules_of_thumb]]
+= RegionServer Sizing Rules of Thumb
+
+Lars Hofhansl wrote a great
+link:http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html[blog post]
+about RegionServer memory sizing. The upshot is that you probably need more memory
+than you think you need. He goes into the impact of region size, memstore size, HDFS
+replication factor, and other things to check.
+
+[quote, Lars Hofhansl, http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html]
+____
+Personally I would place the maximum disk space per machine that can be served
+exclusively with HBase around 6T, unless you have a very read-heavy workload.
+In that case the Java heap should be 32GB (20G regions, 128M memstores, the rest
+defaults).
+____
+
 [[number.of.cfs]]
 ==  On the number of column families
 
@@ -175,7 +231,7 @@ See this comic by IKai Lan on why monotonically increasing row keys are problema
 The pile-up on a single region brought on by monotonically increasing keys can be mitigated by randomizing the input records to not be in sorted order, but in general it's best to avoid using a timestamp or a sequence (e.g. 1, 2, 3) as the row-key.
 
 If you do need to upload time series data into HBase, you should study link:http://opentsdb.net/[OpenTSDB] as a successful example.
-It has a page describing the link: http://opentsdb.net/schema.html[schema] it uses in HBase.
+It has a page describing the link:http://opentsdb.net/schema.html[schema] it uses in HBase.
 The key format in OpenTSDB is effectively [metric_type][event_timestamp], which would appear at first glance to contradict the previous advice about not using a timestamp as the key.
 However, the difference is that the timestamp is not in the _lead_ position of the key, and the design assumption is that there are dozens or hundreds (or more) of different metric types.
 Thus, even with a continual stream of input data with a mix of metric types, the Puts are distributed across various points of regions in the table.
@@ -327,8 +383,8 @@ As an example of why this is important, consider the example of using displayabl
 
 The problem is that all the data is going to pile up in the first 2 regions and the last region thus creating a "lumpy" (and possibly "hot") region problem.
 To understand why, refer to an link:http://www.asciitable.com[ASCII Table].
-'0' is byte 48, and 'f' is byte 102, but there is a huge gap in byte values (bytes 58 to 96) that will _never appear in this keyspace_ because the only values are [0-9] and [a-f]. Thus, the middle regions regions will never be used.
-To make pre-spliting work with this example keyspace, a custom definition of splits (i.e., and not relying on the built-in split method) is required.
+'0' is byte 48, and 'f' is byte 102, but there is a huge gap in byte values (bytes 58 to 96) that will _never appear in this keyspace_ because the only values are [0-9] and [a-f]. Thus, the middle regions will never be used.
+To make pre-splitting work with this example keyspace, a custom definition of splits (i.e., and not relying on the built-in split method) is required.
 
 Lesson #1: Pre-splitting tables is generally a best practice, but you need to pre-split them in such a way that all the regions are accessible in the keyspace.
 While this example demonstrated the problem with a hex-key keyspace, the same problem can happen with _any_ keyspace.
@@ -394,7 +450,7 @@ The minimum number of row versions parameter is used together with the time-to-l
 HBase supports a "bytes-in/bytes-out" interface via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] and link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Result.html[Result], so anything that can be converted to an array of bytes can be stored as a value.
 Input could be strings, numbers, complex objects, or even images as long as they can rendered as bytes.
 
-There are practical limits to the size of values (e.g., storing 10-50MB objects in HBase would probably be too much to ask); search the mailling list for conversations on this topic.
+There are practical limits to the size of values (e.g., storing 10-50MB objects in HBase would probably be too much to ask); search the mailing list for conversations on this topic.
 All rows in HBase conform to the <<datamodel>>, and that includes versioning.
 Take that into consideration when making your design, as well as block size for the ColumnFamily.
 
@@ -502,7 +558,7 @@ ROW                                              COLUMN+CELL
 
 Notice how delete cells are let go.
 
-Now lets run the same test only with `KEEP_DELETED_CELLS` set on the table (you can do table or per-column-family):
+Now let's run the same test only with `KEEP_DELETED_CELLS` set on the table (you can do table or per-column-family):
 
 [source]
 ----
@@ -593,7 +649,7 @@ However, don't try a full-scan on a large table like this from an application (i
 [[secondary.indexes.periodic]]
 ===  Periodic-Update Secondary Index
 
-A secondary index could be created in an other table which is periodically updated via a MapReduce job.
+A secondary index could be created in another table which is periodically updated via a MapReduce job.
 The job could be executed intra-day, but depending on load-strategy it could still potentially be out of sync with the main data table.
 
 See <<mapreduce.example.readwrite,mapreduce.example.readwrite>> for more information.
@@ -615,13 +671,18 @@ See <<mapreduce.example.summary,mapreduce.example.summary>> for more information
 ===  Coprocessor Secondary Index
 
 Coprocessors act like RDBMS triggers. These were added in 0.92.
-For more information, see <<coprocessors,coprocessors>>
+For more information, see <<cp,coprocessors>>
 
 == Constraints
 
 HBase currently supports 'constraints' in traditional (SQL) database parlance.
-The advised usage for Constraints is in enforcing business rules for attributes in the table (e.g. make sure values are in the range 1-10). Constraints could also be used to enforce referential integrity, but this is strongly discouraged as it will dramatically decrease the write throughput of the tables where integrity checking is enabled.
-Extensive documentation on using Constraints can be found at: link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/constraint[Constraint] since version 0.94.
+The advised usage for Constraints is in enforcing business rules for attributes
+in the table (e.g. make sure values are in the range 1-10). Constraints could
+also be used to enforce referential integrity, but this is strongly discouraged
+as it will dramatically decrease the write throughput of the tables where integrity
+checking is enabled. Extensive documentation on using Constraints can be found at
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/constraint/Constraint.html[Constraint]
+since version 0.94.
 
 [[schema.casestudies]]
 == Schema Design Case Studies
@@ -700,7 +761,7 @@ See https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#se
 ====
 
 [[schema.casestudies.log_timeseries.varkeys]]
-==== Variangle Length or Fixed Length Rowkeys?
+==== Variable Length or Fixed Length Rowkeys?
 
 It is critical to remember that rowkeys are stamped on every column in HBase.
 If the hostname is `a` and the event type is `e1` then the resulting rowkey would be quite small.
@@ -721,10 +782,12 @@ Composite Rowkey With Numeric Substitution:
 For this approach another lookup table would be needed in addition to LOG_DATA, called LOG_TYPES.
 The rowkey of LOG_TYPES would be:
 
-* [type] (e.g., byte indicating hostname vs. event-type)
-* [bytes] variable length bytes for raw hostname or event-type.
+* `[type]` (e.g., byte indicating hostname vs. event-type)
+* `[bytes]` variable length bytes for raw hostname or event-type.
 
-A column for this rowkey could be a long with an assigned number, which could be obtained by using an link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29[HBase counter].
+A column for this rowkey could be a long with an assigned number, which could be obtained
+by using an
++++<a href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#incrementColumnValue%28byte[],%20byte[],%20byte[],%20long%29">HBase counter</a>+++.
 
 So the resulting composite rowkey would be:
 
@@ -739,7 +802,9 @@ In either the Hash or Numeric substitution approach, the raw values for hostname
 
 This effectively is the OpenTSDB approach.
 What OpenTSDB does is re-write data and pack rows into columns for certain time-periods.
-For a detailed explanation, see: link:http://opentsdb.net/schema.html, and link:http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/video-hbasecon-2012-lessons-learned-from-opentsdb.html[Lessons Learned from OpenTSDB] from HBaseCon2012.
+For a detailed explanation, see: http://opentsdb.net/schema.html, and
++++<a href="http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/video-hbasecon-2012-lessons-learned-from-opentsdb.html">Lessons Learned from OpenTSDB</a>+++
+from HBaseCon2012.
 
 But this is how the general concept works: data is ingested, for example, in this manner...
 
@@ -784,7 +849,7 @@ Assuming that the combination of customer number and sales order uniquely identi
 [customer number][order number]
 ----
 
-for a ORDER table.
+for an ORDER table.
 However, there are more design decisions to make: are the _raw_ values the best choices for rowkeys?
 
 The same design questions in the Log Data use-case confront us here.
@@ -842,14 +907,14 @@ The ORDER table's rowkey was described above: <<schema.casestudies.custorder,sch
 
 The SHIPPING_LOCATION's composite rowkey would be something like this:
 
-* [order-rowkey]
-* [shipping location number] (e.g., 1st location, 2nd, etc.)
+* `[order-rowkey]`
+* `[shipping location number]` (e.g., 1st location, 2nd, etc.)
 
 The LINE_ITEM table's composite rowkey would be something like this:
 
-* [order-rowkey]
-* [shipping location number] (e.g., 1st location, 2nd, etc.)
-* [line item number] (e.g., 1st lineitem, 2nd, etc.)
+* `[order-rowkey]`
+* `[shipping location number]` (e.g., 1st location, 2nd, etc.)
+* `[line item number]` (e.g., 1st lineitem, 2nd, etc.)
 
 Such a normalized model is likely to be the approach with an RDBMS, but that's not your only option with HBase.
 The cons of such an approach is that to retrieve information about any Order, you will need:
@@ -867,21 +932,21 @@ With this approach, there would exist a single table ORDER that would contain
 
 The Order rowkey was described above: <<schema.casestudies.custorder,schema.casestudies.custorder>>
 
-* [order-rowkey]
-* [ORDER record type]
+* `[order-rowkey]`
+* `[ORDER record type]`
 
 The ShippingLocation composite rowkey would be something like this:
 
-* [order-rowkey]
-* [SHIPPING record type]
-* [shipping location number] (e.g., 1st location, 2nd, etc.)
+* `[order-rowkey]`
+* `[SHIPPING record type]`
+* `[shipping location number]` (e.g., 1st location, 2nd, etc.)
 
 The LineItem composite rowkey would be something like this:
 
-* [order-rowkey]
-* [LINE record type]
-* [shipping location number] (e.g., 1st location, 2nd, etc.)
-* [line item number] (e.g., 1st lineitem, 2nd, etc.)
+* `[order-rowkey]`
+* `[LINE record type]`
+* `[shipping location number]` (e.g., 1st location, 2nd, etc.)
+* `[line item number]` (e.g., 1st lineitem, 2nd, etc.)
 
 [[schema.casestudies.custorder.obj.denorm]]
 ===== Denormalized
@@ -890,9 +955,9 @@ A variant of the Single Table With Record Types approach is to denormalize and f
 
 The LineItem composite rowkey would be something like this:
 
-* [order-rowkey]
-* [LINE record type]
-* [line item number] (e.g., 1st lineitem, 2nd, etc., care must be taken that there are unique across the entire order)
+* `[order-rowkey]`
+* `[LINE record type]`
+* `[line item number]` (e.g., 1st lineitem, 2nd, etc., care must be taken that there are unique across the entire order)
 
 and the LineItem columns would be something like this:
 
@@ -915,9 +980,9 @@ For example, the ORDER table's rowkey was described above: <<schema.casestudies.
 
 There are many options here: JSON, XML, Java Serialization, Avro, Hadoop Writables, etc.
 All of them are variants of the same approach: encode the object graph to a byte-array.
-Care should be taken with this approach to ensure backward compatibilty in case the object model changes such that older persisted structures can still be read back out of HBase.
+Care should be taken with this approach to ensure backward compatibility in case the object model changes such that older persisted structures can still be read back out of HBase.
 
-Pros are being able to manage complex object graphs with minimal I/O (e.g., a single HBase Get per Order in this example), but the cons include the aforementioned warning about backward compatiblity of serialization, language dependencies of serialization (e.g., Java Serialization only works with Java clients), the fact that you have to deserialize the entire object to get any piece of information inside the BLOB, and the difficulty in getting frameworks like Hive to work with custom objects like this.
+Pros are being able to manage complex object graphs with minimal I/O (e.g., a single HBase Get per Order in this example), but the cons include the aforementioned warning about backward compatibility of serialization, language dependencies of serialization (e.g., Java Serialization only works with Java clients), the fact that you have to deserialize the entire object to get any piece of information inside the BLOB, and the difficulty in getting frameworks like Hive to work with custom objects like this.
 
 [[schema.smackdown]]
 === Case Study - "Tall/Wide/Middle" Schema Design Smackdown
@@ -929,7 +994,7 @@ These are general guidelines and not laws - each application must consider its o
 ==== Rows vs. Versions
 
 A common question is whether one should prefer rows or HBase's built-in-versioning.
-The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 1 max versions). The rows-approach would require storing a timestamp in some portion of the rowkey so that they would not overwite with each successive update.
+The context is typically where there are "a lot" of versions of a row to be retained (e.g., where it is significantly above the HBase default of 1 max versions). The rows-approach would require storing a timestamp in some portion of the rowkey so that they would not overwrite with each successive update.
 
 Preference: Rows (generally speaking).
 
@@ -1028,14 +1093,14 @@ The tl;dr version is that you should probably go with one row per user+value, an
 
 Your two options mirror a common question people have when designing HBase schemas: should I go "tall" or "wide"? Your first schema is "tall": each row represents one value for one user, and so there are many rows in the table for each user; the row key is user + valueid, and there would be (presumably) a single column qualifier that means "the value". This is great if you want to scan over rows in sorted order by row key (thus my question above, about whether these ids are sorted correctly). You can start a scan at any user+valueid, read the next 30, and be done.
 What you're giving up is the ability to have transactional guarantees around all the rows for one user, but it doesn't sound like you need that.
-Doing it this way is generally recommended (see here link:http://hbase.apache.org/book.html#schema.smackdown).
+Doing it this way is generally recommended (see here http://hbase.apache.org/book.html#schema.smackdown).
 
 Your second option is "wide": you store a bunch of values in one row, using different qualifiers (where the qualifier is the valueid). The simple way to do that would be to just store ALL values for one user in a single row.
 I'm guessing you jumped to the "paginated" version because you're assuming that storing millions of columns in a single row would be bad for performance, which may or may not be true; as long as you're not trying to do too much in a single request, or do things like scanning over and returning all of the cells in the row, it shouldn't be fundamentally worse.
 The client has methods that allow you to get specific slices of columns.
 
 Note that neither case fundamentally uses more disk space than the other; you're just "shifting" part of the identifying information for a value either to the left (into the row key, in option one) or to the right (into the column qualifiers in option 2). Under the covers, every key/value still stores the whole row key, and column family name.
-(If this is a bit confusing, take an hour and watch Lars George's excellent video about understanding HBase schema design: link:http://www.youtube.com/watch?v=_HLoH_PgrLk).
+(If this is a bit confusing, take an hour and watch Lars George's excellent video about understanding HBase schema design: http://www.youtube.com/watch?v=_HLoH_PgrLk).
 
 A manually paginated version has lots more complexities, as you note, like having to keep track of how many things are in each page, re-shuffling if new values are inserted, etc.
 That seems significantly more complex.
@@ -1045,4 +1110,101 @@ If you don't have time to build it both ways and compare, my advice would be to
 [[schema.ops]]
 == Operational and Performance Configuration Options
 
-See the Performance section <<perf.schema,perf.schema>> for more information operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.
+====  Tune HBase Server RPC Handling
+
+* Set `hbase.regionserver.handler.count` (in `hbase-site.xml`) to cores x spindles for concurrency.
+* Optionally, split the call queues into separate read and write queues for differentiated service. The parameter `hbase.ipc.server.callqueue.handler.factor` specifies the number of call queues:
+- `0` means a single shared queue
+- `1` means one queue for each handler.
+- A value between `0` and `1` allocates the number of queues proportionally to the number of handlers. For instance, a value of `.5` shares one queue between each two handlers.
+* Use `hbase.ipc.server.callqueue.read.ratio` (`hbase.ipc.server.callqueue.read.share` in 0.98) to split the call queues into read and write queues:
+- `0.5` means there will be the same number of read and write queues
+- `< 0.5` for more read than write
+- `> 0.5` for more write than read
+* Set `hbase.ipc.server.callqueue.scan.ratio` (HBase 1.0+)  to split read call queues into small-read and long-read queues:
+- 0.5 means that there will be the same number of short-read and long-read queues
+- `< 0.5` for more short-read
+- `> 0.5` for more long-read
+
+====  Disable Nagle for RPC
+
+Disable Nagle\u2019s algorithm. Delayed ACKs can add up to ~200ms to RPC round trip time. Set the following parameters:
+
+* In Hadoop\u2019s `core-site.xml`:
+- `ipc.server.tcpnodelay = true`
+- `ipc.client.tcpnodelay = true`
+* In HBase\u2019s `hbase-site.xml`:
+- `hbase.ipc.client.tcpnodelay = true`
+- `hbase.ipc.server.tcpnodelay = true`
+
+====  Limit Server Failure Impact
+
+Detect regionserver failure as fast as reasonable. Set the following parameters:
+
+* In `hbase-site.xml`, set `zookeeper.session.timeout` to 30 seconds or less to bound failure detection (20-30 seconds is a good start).
+* Detect and avoid unhealthy or failed HDFS DataNodes: in `hdfs-site.xml` and `hbase-site.xml`, set the following parameters:
+- `dfs.namenode.avoid.read.stale.datanode = true`
+- `dfs.namenode.avoid.write.stale.datanode = true`
+
+====  Optimize on the Server Side for Low Latency
+
+* Skip the network for local blocks. In `hbase-site.xml`, set the following parameters:
+- `dfs.client.read.shortcircuit = true`
+- `dfs.client.read.shortcircuit.buffer.size = 131072` (Important to avoid OOME)
+* Ensure data locality. In `hbase-site.xml`, set `hbase.hstore.min.locality.to.skip.major.compact = 0.7` (Meaning that 0.7 \<= n \<= 1)
+* Make sure DataNodes have enough handlers for block transfers. In `hdfs-site`.xml``, set the following parameters:
+- `dfs.datanode.max.xcievers >= 8192`
+- `dfs.datanode.handler.count =` number of spindles
+
+===  JVM Tuning
+
+====  Tune JVM GC for low collection latencies
+
+* Use the CMS collector: `-XX:+UseConcMarkSweepGC`
+* Keep eden space as small as possible to minimize average collection time. Example:
+
+    -XX:CMSInitiatingOccupancyFraction=70
+
+* Optimize for low collection latency rather than throughput: `-Xmn512m`
+* Collect eden in parallel: `-XX:+UseParNewGC`
+*  Avoid collection under pressure: `-XX:+UseCMSInitiatingOccupancyOnly`
+* Limit per request scanner result sizing so everything fits into survivor space but doesn\u2019t tenure. In `hbase-site.xml`, set `hbase.client.scanner.max.result.size` to 1/8th of eden space (with -`Xmn512m` this is ~51MB )
+* Set `max.result.size` x `handler.count` less than survivor space
+
+====  OS-Level Tuning
+
+* Turn transparent huge pages (THP) off:
+
+  echo never > /sys/kernel/mm/transparent_hugepage/enabled
+  echo never > /sys/kernel/mm/transparent_hugepage/defrag
+
+* Set `vm.swappiness = 0`
+* Set `vm.min_free_kbytes` to at least 1GB (8GB on larger memory systems)
+* Disable NUMA zone reclaim with `vm.zone_reclaim_mode = 0`
+
+==  Special Cases
+
+====  For applications where failing quickly is better than waiting
+
+*  In `hbase-site.xml` on the client side, set the following parameters:
+- Set `hbase.client.pause = 1000`
+- Set `hbase.client.retries.number = 3`
+- If you want to ride over splits and region moves, increase `hbase.client.retries.number` substantially (>= 20)
+- Set the RecoverableZookeeper retry count: `zookeeper.recovery.retry = 1` (no retry)
+* In `hbase-site.xml` on the server side, set the Zookeeper session timeout for detecting server failures: `zookeeper.session.timeout` <= 30 seconds (20-30 is good).
+
+====  For applications that can tolerate slightly out of date information
+
+**HBase timeline consistency (HBASE-10070) **
+With read replicas enabled, read-only copies of regions (replicas) are distributed over the cluster. One RegionServer services the default or primary replica, which is the only replica that can service writes. Other RegionServers serve the secondary replicas, follow the primary RegionServer, and only see committed updates. The secondary replicas are read-only, but can serve reads immediately while the primary is failing over, cutting read availability blips from seconds to milliseconds. Phoenix supports timeline consistency as of 4.4.0
+Tips:
+
+* Deploy HBase 1.0.0 or later.
+* Enable timeline consistent replicas on the server side.
+* Use one of the following methods to set timeline consistency:
+- Use `ALTER SESSION SET CONSISTENCY = 'TIMELINE\u2019`
+- Set the connection property `Consistency` to `timeline` in the JDBC connect string
+
+=== More Information
+
+See the Performance section <<perf.schema,perf.schema>> for more information about operational and performance schema design options, such as Bloom Filters, Table-configured regionsizes, compression, and blocksizes.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index 101affa..0ed9ba2 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -42,7 +42,7 @@ HBase provides mechanisms to secure various components and aspects of HBase and
 == Using Secure HTTP (HTTPS) for the Web UI
 
 A default HBase install uses insecure HTTP connections for Web UIs for the master and region servers.
-To enable secure HTTP (HTTPS) connections instead, set `hadoop.ssl.enabled` to `true` in _hbase-site.xml_.
+To enable secure HTTP (HTTPS) connections instead, set `hbase.ssl.enabled` to `true` in _hbase-site.xml_.
 This does not change the port used by the Web UI.
 To change the port for the web UI for a given HBase component, configure that port's setting in hbase-site.xml.
 These settings are:
@@ -69,11 +69,64 @@ See Nick Dimiduk's contribution on this link:http://stackoverflow.com/questions/
 If you know how to fix this without opening a second port for HTTPS, patches are appreciated.
 ====
 
+[[hbase.secure.spnego.ui]]
+== Using SPNEGO for Kerberos authentication with Web UIs
+
+Kerberos-authentication to HBase Web UIs can be enabled via configuring SPNEGO with the `hbase.security.authentication.ui`
+property in _hbase-site.xml_. Enabling this authentication requires that HBase is also configured to use Kerberos authentication
+for RPCs (e.g `hbase.security.authentication` = `kerberos`).
+
+[source,xml]
+----
+<property>
+  <name>hbase.security.authentication.ui</name>
+  <value>kerberos</value>
+  <description>Controls what kind of authentication should be used for the HBase web UIs.</description>
+</property>
+<property>
+  <name>hbase.security.authentication</name>
+  <value>kerberos</value>
+  <description>The Kerberos keytab file to use for SPNEGO authentication by the web server.</description>
+</property>
+----
+
+A number of properties exist to configure SPNEGO authentication for the web server:
+
+[source,xml]
+----
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.principal</name>
+  <value>HTTP/_HOST@EXAMPLE.COM</value>
+  <description>Required for SPNEGO, the Kerberos principal to use for SPNEGO authentication by the
+  web server. The _HOST keyword will be automatically substituted with the node's
+  hostname.</description>
+</property>
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.keytab</name>
+  <value>/etc/security/keytabs/spnego.service.keytab</value>
+  <description>Required for SPNEGO, the Kerberos keytab file to use for SPNEGO authentication by the
+  web server.</description>
+</property>
+<property>
+  <name>hbase.security.authentication.spnego.kerberos.name.rules</name>
+  <value></value>
+  <description>Optional, Hadoop-style `auth_to_local` rules which will be parsed and used in the
+  handling of Kerberos principals</description>
+</property>
+<property>
+  <name>hbase.security.authentication.signature.secret.file</name>
+  <value></value>
+  <description>Optional, a file whose contents will be used as a secret to sign the HTTP cookies
+  as a part of the SPNEGO authentication handshake. If this is not provided, Java's `Random` library
+  will be used for the secret.</description>
+</property>
+----
+
 [[hbase.secure.configuration]]
 == Secure Client Access to Apache HBase
 
 Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
-See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
+See also Matteo Bertozzi's article on link:https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
 
 This describes how to set up Apache HBase and clients for connection to secure HBase resources.
 
@@ -175,6 +228,15 @@ Add the following to the `hbase-site.xml` file for every Thrift gateway:
    You may have  to put the concrete full hostname.
    -->
 </property>
+<!-- Add these if you need to configure a different DNS interface from the default -->
+<property>
+  <name>hbase.thrift.dns.interface</name>
+  <value>default</value>
+</property>
+<property>
+  <name>hbase.thrift.dns.nameserver</name>
+  <value>default</value>
+</property>
 ----
 
 Substitute the appropriate credential and keytab for _$USER_ and _$KEYTAB_ respectively.
@@ -227,39 +289,41 @@ To enable it, do the following.
 
 <<security.gateway.thrift>> describes how to configure the Thrift gateway to authenticate to HBase on the client's behalf, and to access HBase using a proxy user. The limitation of this approach is that after the client is initialized with a particular set of credentials, it cannot change these credentials during the session. The `doAs` feature provides a flexible way to impersonate multiple principals using the same client. This feature was implemented in link:https://issues.apache.org/jira/browse/HBASE-12640[HBASE-12640] for Thrift 1, but is currently not available for Thrift 2.
 
-*To allow proxy users*, add the following to the _hbase-site.xml_ file for every HBase node:
+*To enable the `doAs` feature*, add the following to the _hbase-site.xml_ file for every Thrift gateway:
 
 [source,xml]
 ----
 <property>
-  <name>hadoop.security.authorization</name>
+  <name>hbase.regionserver.thrift.http</name>
   <value>true</value>
 </property>
 <property>
-  <name>hadoop.proxyuser.$USER.groups</name>
-  <value>$GROUPS</value>
-</property>
-<property>
-  <name>hadoop.proxyuser.$USER.hosts</name>
-  <value>$GROUPS</value>
+  <name>hbase.thrift.support.proxyuser</name>
+  <value>true/value>
 </property>
 ----
 
-*To enable the `doAs` feature*, add the following to the _hbase-site.xml_ file for every Thrift gateway:
+*To allow proxy users* when using `doAs` impersonation, add the following to the _hbase-site.xml_ file for every HBase node:
 
 [source,xml]
 ----
 <property>
-  <name>hbase.regionserver.thrift.http</name>
+  <name>hadoop.security.authorization</name>
   <value>true</value>
 </property>
 <property>
-  <name>hbase.thrift.support.proxyuser</name>
-  <value>true/value>
+  <name>hadoop.proxyuser.$USER.groups</name>
+  <value>$GROUPS</value>
+</property>
+<property>
+  <name>hadoop.proxyuser.$USER.hosts</name>
+  <value>$GROUPS</value>
 </property>
 ----
 
-Take a look at the link:https://github.com/apache/hbase/blob/master/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java[demo client] to get an overall idea of how to use this feature in your client.
+Take a look at the
+link:https://github.com/apache/hbase/blob/master/hbase-examples/src/main/java/org/apache/hadoop/hbase/thrift/HttpDoAsClient.java[demo client]
+to get an overall idea of how to use this feature in your client.
 
 === Client-side Configuration for Secure Operation - REST Gateway
 
@@ -297,6 +361,10 @@ To enable REST gateway Kerberos authentication for client access, add the follow
 [source,xml]
 ----
 <property>
+  <name>hbase.rest.support.proxyuser</name>
+  <value>true</value>
+</property>
+<property>
   <name>hbase.rest.authentication.type</name>
   <value>kerberos</value>
 </property>
@@ -308,12 +376,21 @@ To enable REST gateway Kerberos authentication for client access, add the follow
   <name>hbase.rest.authentication.kerberos.keytab</name>
   <value>$KEYTAB</value>
 </property>
+<!-- Add these if you need to configure a different DNS interface from the default -->
+<property>
+  <name>hbase.rest.dns.interface</name>
+  <value>default</value>
+</property>
+<property>
+  <name>hbase.rest.dns.nameserver</name>
+  <value>default</value>
+</property>
 ----
 
 Substitute the keytab for HTTP for _$KEYTAB_.
 
 HBase REST gateway supports different 'hbase.rest.authentication.type': simple, kerberos.
-You can also implement a custom authentication by implemening Hadoop AuthenticationHandler, then specify the full class name as 'hbase.rest.authentication.type' value.
+You can also implement a custom authentication by implementing Hadoop AuthenticationHandler, then specify the full class name as 'hbase.rest.authentication.type' value.
 For more information, refer to link:http://hadoop.apache.org/docs/stable/hadoop-auth/index.html[SPNEGO HTTP authentication].
 
 [[security.rest.gateway]]
@@ -325,7 +402,7 @@ To the HBase server, all requests are from the REST gateway user.
 The actual users are unknown.
 You can turn on the impersonation support.
 With impersonation, the REST gateway user is a proxy user.
-The HBase server knows the acutal/real user of each request.
+The HBase server knows the actual/real user of each request.
 So it can apply proper authorizations.
 
 To turn on REST gateway impersonation, we need to configure HBase servers (masters and region servers) to allow proxy users; configure REST gateway to enable impersonation.
@@ -374,7 +451,7 @@ Substitute the keytab for HTTP for _$KEYTAB_.
 == Simple User Access to Apache HBase
 
 Newer releases of Apache HBase (>= 0.92) support optional SASL authentication of clients.
-See also Matteo Bertozzi's article on link:http://www.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
+See also Matteo Bertozzi's article on link:https://blog.cloudera.com/blog/2012/09/understanding-user-authentication-and-authorization-in-apache-hbase/[Understanding User Authentication and Authorization in Apache HBase].
 
 This describes how to set up Apache HBase and clients for simple user access to HBase resources.
 
@@ -504,21 +581,21 @@ This is future work.
 Secure HBase requires secure ZooKeeper and HDFS so that users cannot access and/or modify the metadata and data from under HBase. HBase uses HDFS (or configured file system) to keep its data files as well as write ahead logs (WALs) and other data. HBase uses ZooKeeper to store some metadata for operations (master address, table locks, recovery state, etc).
 
 === Securing ZooKeeper Data
-ZooKeeper has a pluggable authentication mechanism to enable access from clients using different methods. ZooKeeper even allows authenticated and un-authenticated clients at the same time. The access to znodes can be restricted by providing Access Control Lists (ACLs) per znode. An ACL contains two components, the authentication method and the principal. ACLs are NOT enforced hierarchically. See link:https://zookeeper.apache.org/doc/r3.3.6/zookeeperProgrammers.html#sc_ZooKeeperPluggableAuthentication[ZooKeeper Programmers Guide] for details. 
+ZooKeeper has a pluggable authentication mechanism to enable access from clients using different methods. ZooKeeper even allows authenticated and un-authenticated clients at the same time. The access to znodes can be restricted by providing Access Control Lists (ACLs) per znode. An ACL contains two components, the authentication method and the principal. ACLs are NOT enforced hierarchically. See link:https://zookeeper.apache.org/doc/r3.3.6/zookeeperProgrammers.html#sc_ZooKeeperPluggableAuthentication[ZooKeeper Programmers Guide] for details.
 
-HBase daemons authenticate to ZooKeeper via SASL and kerberos (See <<zk.sasl.auth>>). HBase sets up the znode ACLs so that only the HBase user and the configured hbase superuser (`hbase.superuser`) can access and modify the data. In cases where ZooKeeper is used for service discovery or sharing state with the client, the znodes created by HBase will also allow anyone (regardless of authentication) to read these znodes (clusterId, master address, meta location, etc), but only the HBase user can modify them. 
+HBase daemons authenticate to ZooKeeper via SASL and kerberos (See <<zk.sasl.auth>>). HBase sets up the znode ACLs so that only the HBase user and the configured hbase superuser (`hbase.superuser`) can access and modify the data. In cases where ZooKeeper is used for service discovery or sharing state with the client, the znodes created by HBase will also allow anyone (regardless of authentication) to read these znodes (clusterId, master address, meta location, etc), but only the HBase user can modify them.
 
 === Securing File System (HDFS) Data
-All of the data under management is kept under the root directory in the file system (`hbase.rootdir`). Access to the data and WAL files in the filesystem should be restricted so that users cannot bypass the HBase layer, and peek at the underlying data files from the file system. HBase assumes the filesystem used (HDFS or other) enforces permissions hierarchically. If sufficient protection from the file system (both authorization and authentication) is not provided, HBase level authorization control (ACLs, visibility labels, etc) is meaningless since the user can always access the data from the file system. 
+All of the data under management is kept under the root directory in the file system (`hbase.rootdir`). Access to the data and WAL files in the filesystem should be restricted so that users cannot bypass the HBase layer, and peek at the underlying data files from the file system. HBase assumes the filesystem used (HDFS or other) enforces permissions hierarchically. If sufficient protection from the file system (both authorization and authentication) is not provided, HBase level authorization control (ACLs, visibility labels, etc) is meaningless since the user can always access the data from the file system.
 
 HBase enforces the posix-like permissions 700 (`rwx------`) to its root directory. It means that only the HBase user can read or write the files in FS. The default setting can be changed by configuring `hbase.rootdir.perms` in hbase-site.xml. A restart of the active master is needed so that it changes the used permissions. For versions before 1.2.0, you can check whether HBASE-13780 is committed, and if not, you can manually set the permissions for the root directory if needed. Using HDFS, the command would be:
 [source,bash]
 ----
 sudo -u hdfs hadoop fs -chmod 700 /hbase
 ----
-You should change `/hbase` if you are using a different `hbase.rootdir`. 
+You should change `/hbase` if you are using a different `hbase.rootdir`.
 
-In secure mode, SecureBulkLoadEndpoint should be configured and used for properly handing of users files created from MR jobs to the HBase daemons and HBase user. The staging directory in the distributed file system used for bulk load (`hbase.bulkload.staging.dir`, defaults to `/tmp/hbase-staging`) should have (mode 711, or `rwx--x--x`) so that users can access the staging directory created under that parent directory, but cannot do any other operation. See <<hbase.secure.bulkload>> for how to configure SecureBulkLoadEndPoint. 
+In secure mode, SecureBulkLoadEndpoint should be configured and used for properly handing of users files created from MR jobs to the HBase daemons and HBase user. The staging directory in the distributed file system used for bulk load (`hbase.bulkload.staging.dir`, defaults to `/tmp/hbase-staging`) should have (mode 711, or `rwx--x--x`) so that users can access the staging directory created under that parent directory, but cannot do any other operation. See <<hbase.secure.bulkload>> for how to configure SecureBulkLoadEndPoint.
 
 == Securing Access To Your Data
 
@@ -548,6 +625,7 @@ Several procedures in this section require you to copy files between cluster nod
 When copying keys, configuration files, or other files containing sensitive strings, use a secure method, such as `ssh`, to avoid leaking sensitive data.
 ====
 
+[[security.data.basic.server.side]]
 .Procedure: Basic Server-Side Configuration
 . Enable HFile v3, by setting `hfile.format.version` to 3 in _hbase-site.xml_.
   This is the default for HBase 1.0 and newer.
@@ -1044,7 +1122,7 @@ public static void verifyAllowed(User user, AccessTestAction action, int count)
 ----
 ====
 
-
+[[hbase.visibility.labels]]
 === Visibility Labels
 
 Visibility labels control can be used to only permit users or principals associated with a given label to read or access cells with that label.
@@ -1099,7 +1177,7 @@ NOTE: Visibility labels are not currently applied for superusers.
 | Interpretation
 
 | fulltime
-| Allow accesss to users associated with the fulltime label.
+| Allow access to users associated with the fulltime label.
 
 | !public
 | Allow access to users not associated with the public label.
@@ -1314,11 +1392,21 @@ static Table createTableAndWriteDataWithLabels(TableName tableName, String... la
 ----
 ====
 
-<<reading_cells_with_labels>>
+[[reading_cells_with_labels]]
 ==== Reading Cells with Labels
-When you issue a Scan or Get, HBase uses your default set of authorizations to filter out cells that you do not have access to. A superuser can set the default set of authorizations for a given user by using the `set_auths` HBase Shell command or the link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityClient.html#setAuths(org.apache.hadoop.conf.Configuration,%20java.lang.String\[\],%20java.lang.String)[VisibilityClient.setAuths()] method.
 
-You can specify a different authorization during the Scan or Get, by passing the AUTHORIZATIONS option in HBase Shell, or the link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setAuthorizations%28org.apache.hadoop.hbase.security.visibility.Authorizations%29[setAuthorizations()] method if you use the API. This authorization will be combined with your default set as an additional filter. It will further filter your results, rather than giving you additional authorization.
+When you issue a Scan or Get, HBase uses your default set of authorizations to
+filter out cells that you do not have access to. A superuser can set the default
+set of authorizations for a given user by using the `set_auths` HBase Shell command
+or the
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityClient.html#setAuths(org.apache.hadoop.hbase.client.Connection,%20java.lang.String\[\],%20java.lang.String)[VisibilityClient.setAuths()] method.
+
+You can specify a different authorization during the Scan or Get, by passing the
+AUTHORIZATIONS option in HBase Shell, or the
+link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setAuthorizations%28org.apache.hadoop.hbase.security.visibility.Authorizations%29[setAuthorizations()]
+method if you use the API. This authorization will be combined with your default
+set as an additional filter. It will further filter your results, rather than
+giving you additional authorization.
 
 .HBase Shell
 ====
@@ -1564,7 +1652,10 @@ Rotate the Master Key::
 === Secure Bulk Load
 
 Bulk loading in secure mode is a bit more involved than normal setup, since the client has to transfer the ownership of the files generated from the MapReduce job to HBase.
-Secure bulk loading is implemented by a coprocessor, named link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.html[SecureBulkLoadEndpoint], which uses a staging directory configured by the configuration property `hbase.bulkload.staging.dir`, which defaults to _/tmp/hbase-staging/_.
+Secure bulk loading is implemented by a coprocessor, named
+link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.html[SecureBulkLoadEndpoint],
+which uses a staging directory configured by the configuration property `hbase.bulkload.staging.dir`, which defaults to
+_/tmp/hbase-staging/_.
 
 .Secure Bulk Load Algorithm
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/shell.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/shell.adoc b/src/main/asciidoc/_chapters/shell.adoc
index 237089e..8f1f59b 100644
--- a/src/main/asciidoc/_chapters/shell.adoc
+++ b/src/main/asciidoc/_chapters/shell.adoc
@@ -76,7 +76,7 @@ NOTE: Spawning HBase Shell commands in this way is slow, so keep that in mind wh
 
 .Passing Commands to the HBase Shell
 ====
-You can pass commands to the HBase Shell in non-interactive mode (see <<hbasee.shell.noninteractive,hbasee.shell.noninteractive>>) using the `echo` command and the `|` (pipe) operator.
+You can pass commands to the HBase Shell in non-interactive mode (see <<hbase.shell.noninteractive,hbase.shell.noninteractive>>) using the `echo` command and the `|` (pipe) operator.
 Be sure to escape characters in the HBase commands which would otherwise be interpreted by the shell.
 Some debug-level output has been truncated from the example below.
 
@@ -352,6 +352,68 @@ hbase(main):022:0> Date.new(1218920189000).toString() => "Sat Aug 16 20:56:29 UT
 
 To output in a format that is exactly like that of the HBase log format will take a little messing with link:http://download.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html[SimpleDateFormat].
 
+[[tricks.pre-split]]
+=== Pre-splitting tables with the HBase Shell
+You can use a variety of options to pre-split tables when creating them via the HBase Shell `create` command.
+
+The simplest approach is to specify an array of split points when creating the table. Note that when specifying string literals as split points, these will create split points based on the underlying byte representation of the string. So when specifying a split point of '10', we are actually specifying the byte split point '\x31\30'.
+
+The split points will define `n+1` regions where `n` is the number of split points. The lowest region will contain all keys from the lowest possible key up to but not including the first split point key.
+The next region will contain keys from the first split point up to, but not including the next split point key.
+This will continue for all split points up to the last. The last region will be defined from the last split point up to the maximum possible key.
+
+[source]
+----
+hbase>create 't1','f',SPLITS => ['10','20',30']
+----
+
+In the above example, the table 't1' will be created with column family 'f', pre-split to four regions. Note the first region will contain all keys from '\x00' up to '\x30' (as '\x31' is the ASCII code for '1').
+
+You can pass the split points in a file using following variation. In this example, the splits are read from a file corresponding to the local path on the local filesystem. Each line in the file specifies a split point key.
+
+[source]
+----
+hbase>create 't14','f',SPLITS_FILE=>'splits.txt'
+----
+
+The other options are to automatically compute splits based on a desired number of regions and a splitting algorithm.
+HBase supplies algorithms for splitting the key range based on uniform splits or based on hexadecimal keys, but you can provide your own splitting algorithm to subdivide the key range.
+
+[source]
+----
+# create table with four regions based on random bytes keys
+hbase>create 't2','f1', { NUMREGIONS => 4 , SPLITALGO => 'UniformSplit' }
+
+# create table with five regions based on hex keys
+hbase>create 't3','f1', { NUMREGIONS => 5, SPLITALGO => 'HexStringSplit' }
+----
+
+As the HBase Shell is effectively a Ruby environment, you can use simple Ruby scripts to compute splits algorithmically.
+
+[source]
+----
+# generate splits for long (Ruby fixnum) key range from start to end key
+hbase(main):070:0> def gen_splits(start_key,end_key,num_regions)
+hbase(main):071:1>   results=[]
+hbase(main):072:1>   range=end_key-start_key
+hbase(main):073:1>   incr=(range/num_regions).floor
+hbase(main):074:1>   for i in 1 .. num_regions-1
+hbase(main):075:2>     results.push([i*incr+start_key].pack("N"))
+hbase(main):076:2>   end
+hbase(main):077:1>   return results
+hbase(main):078:1> end
+hbase(main):079:0>
+hbase(main):080:0> splits=gen_splits(1,2000000,10)
+=> ["\000\003\r@", "\000\006\032\177", "\000\t'\276", "\000\f4\375", "\000\017B<", "\000\022O{", "\000\025\\\272", "\000\030i\371", "\000\ew8"]
+hbase(main):081:0> create 'test_splits','f',SPLITS=>splits
+0 row(s) in 0.2670 seconds
+
+=> Hbase::Table - test_splits
+----
+
+Note that the HBase Shell command `truncate` effectively drops and recreates the table with default options which will discard any pre-splitting.
+If you need to truncate a pre-split table, you must drop and recreate the table explicitly to re-specify custom split options.
+
 === Debug
 
 ==== Shell debug switch


[6/8] hbase git commit: HBASE-15347 updated asciidoc for 1.3

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index 01f2eb7..b4c39c8 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -28,7 +28,9 @@
 :experimental:
 
 This chapter expands upon the <<getting_started>> chapter to further explain configuration of Apache HBase.
-Please read this chapter carefully, especially the <<basic.prerequisites,Basic Prerequisites>> to ensure that your HBase testing and deployment goes smoothly, and prevent data loss.
+Please read this chapter carefully, especially the <<basic.prerequisites,Basic Prerequisites>>
+to ensure that your HBase testing and deployment goes smoothly, and prevent data loss.
+Familiarize yourself with <<hbase_supported_tested_definitions>> as well.
 
 == Configuration Files
 Apache HBase uses the same configuration system as Apache Hadoop.
@@ -98,6 +100,22 @@ This section lists required services and some required system configuration.
 |JDK 7
 |JDK 8
 
+|2.0
+|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
+|link:http://search-hadoop.com/m/YGbbsPxZ723m3as[Not Supported]
+|yes
+
+|1.3
+|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
+|yes
+|yes
+
+
+|1.2
+|link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
+|yes
+|yes
+
 |1.1
 |link:http://search-hadoop.com/m/DHED4Zlz0R1[Not Supported]
 |yes
@@ -116,11 +134,6 @@ deprecated `remove()` method of the `PoolMap` class and is under consideration.
 link:https://issues.apache.org/jira/browse/HBASE-7608[HBASE-7608] for more information about JDK 8
 support.
 
-|0.96
-|yes
-|yes
-|N/A
-
 |0.94
 |yes
 |yes
@@ -129,6 +142,7 @@ support.
 
 NOTE: In HBase 0.98.5 and newer, you must set `JAVA_HOME` on each node of your cluster. _hbase-env.sh_ provides a handy mechanism to do this.
 
+[[os]]
 .Operating System Utilities
 ssh::
   HBase uses the Secure Shell (ssh) command and utilities extensively to communicate between cluster nodes. Each server in the cluster must be running `ssh` so that the Hadoop and HBase daemons can be managed. You must be able to connect to all nodes via SSH, including the local node, from the Master as well as any backup Master, using a shared key rather than a password. You can see the basic methodology for such a set-up in Linux or Unix systems at "<<passwordless.ssh.quickstart>>". If your cluster nodes use OS X, see the section, link:http://wiki.apache.org/hadoop/Running_Hadoop_On_OS_X_10.5_64-bit_%28Single-Node_Cluster%29[SSH: Setting up Remote Desktop and Enabling Self-Login] on the Hadoop wiki.
@@ -143,6 +157,7 @@ Loopback IP::
 NTP::
   The clocks on cluster nodes should be synchronized. A small amount of variation is acceptable, but larger amounts of skew can cause erratic and unexpected behavior. Time synchronization is one of the first things to check if you see unexplained problems in your cluster. It is recommended that you run a Network Time Protocol (NTP) service, or another time-synchronization mechanism, on your cluster, and that all nodes look to the same service for time synchronization. See the link:http://www.tldp.org/LDP/sag/html/basic-ntp-config.html[Basic NTP Configuration] at [citetitle]_The Linux Documentation Project (TLDP)_ to set up NTP.
 
+[[ulimit]]
 Limits on Number of Files and Processes (ulimit)::
   Apache HBase is a database. It requires the ability to open a large number of files at once. Many Linux distributions limit the number of files a single user is allowed to open to `1024` (or `256` on older versions of OS X). You can check this limit on your servers by running the command `ulimit -n` when logged in as the user which runs HBase. See <<trouble.rs.runtime.filehandles,the Troubleshooting section>> for some of the problems you may experience if the limit is too low. You may also notice errors such as the following:
 +
@@ -162,7 +177,7 @@ For example, assuming that a schema had 3 ColumnFamilies per region with an aver
 +
 Another related setting is the number of processes a user is allowed to run at once. In Linux and Unix, the number of processes is set using the `ulimit -u` command. This should not be confused with the `nproc` command, which controls the number of CPUs available to a given user. Under load, a `ulimit -u` that is too low can cause OutOfMemoryError exceptions. See Jack Levin's major HDFS issues thread on the hbase-users mailing list, from 2011.
 +
-Configuring the maximum number of file descriptors and processes for the user who is running the HBase process is an operating system configuration, rather than an HBase configuration. It is also important to be sure that the settings are changed for the user that actually runs HBase. To see which user started HBase, and that user's ulimit configuration, look at the first line of the HBase log for that instance. A useful read setting config on you hadoop cluster is Aaron Kimballs' Configuration Parameters: What can you just ignore?
+Configuring the maximum number of file descriptors and processes for the user who is running the HBase process is an operating system configuration, rather than an HBase configuration. It is also important to be sure that the settings are changed for the user that actually runs HBase. To see which user started HBase, and that user's ulimit configuration, look at the first line of the HBase log for that instance. A useful read setting config on your hadoop cluster is Aaron Kimball's Configuration Parameters: What can you just ignore?
 +
 .`ulimit` Settings on Ubuntu
 ====
@@ -210,24 +225,39 @@ Use the following legend to interpret this table:
 * "X" = not supported
 * "NT" = Not tested
 
-[cols="1,1,1,1,1,1,1", options="header"]
+[cols="1,1,1,1,1,1,1,1", options="header"]
 |===
-| | HBase-0.92.x | HBase-0.94.x | HBase-0.96.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported) | HBase-1.1.x
-|Hadoop-0.20.205 | S | X | X | X | X | X
-|Hadoop-0.22.x | S | X | X | X | X | X
-|Hadoop-1.0.x  |X | X | X | X | X | X
-|Hadoop-1.1.x | NT | S | S | NT | X | X
-|Hadoop-0.23.x | X | S | NT | X | X | X
-|Hadoop-2.0.x-alpha | X | NT | X | X | X | X
-|Hadoop-2.1.0-beta | X | NT | S | X | X | X
-|Hadoop-2.2.0 | X | NT | S | S | NT | NT
-|Hadoop-2.3.x | X | NT | S | S | NT | NT
-|Hadoop-2.4.x | X | NT | S | S | S | S
-|Hadoop-2.5.x | X | NT | S | S | S | S
-|Hadoop-2.6.x | X | NT | NT | NT | S | S
-|Hadoop-2.7.x | X | NT | NT | NT | NT | NT
+| | HBase-0.94.x | HBase-0.98.x (Support for Hadoop 1.1+ is deprecated.) | HBase-1.0.x (Hadoop 1.x is NOT supported) | HBase-1.1.x | HBase-1.2.x | HBase-1.3.x | HBase-2.0.x
+|Hadoop-1.0.x  | X | X | X | X | X | X | X
+|Hadoop-1.1.x | S | NT | X | X | X | X | X
+|Hadoop-0.23.x | S | X | X | X | X | X | X
+|Hadoop-2.0.x-alpha | NT | X | X | X | X | X | X
+|Hadoop-2.1.0-beta | NT | X | X | X | X | X | X
+|Hadoop-2.2.0 | NT | S | NT | NT | X  | X | X
+|Hadoop-2.3.x | NT | S | NT | NT | X  | X | X
+|Hadoop-2.4.x | NT | S | S | S | S | S | X
+|Hadoop-2.5.x | NT | S | S | S | S | S | X
+|Hadoop-2.6.0 | X | X | X | X | X | X | X
+|Hadoop-2.6.1+ | NT | NT | NT | NT | S | S | S
+|Hadoop-2.7.0 | X | X | X | X | X | X | X
+|Hadoop-2.7.1+ | NT | NT | NT | NT | S | S | S
 |===
 
+.Hadoop 2.6.x
+[TIP]
+====
+Hadoop distributions based on the 2.6.x line *must* have
+link:https://issues.apache.org/jira/browse/HADOOP-11710[HADOOP-11710] applied if you plan to run
+HBase on top of an HDFS Encryption Zone. Failure to do so will result in cluster failure and
+data loss. This patch is present in Apache Hadoop releases 2.6.1+.
+====
+
+.Hadoop 2.7.x
+[TIP]
+====
+Hadoop version 2.7.0 is not tested or supported as the Hadoop PMC has explicitly labeled that release as not being stable.
+====
+
 .Replace the Hadoop Bundled With HBase!
 [NOTE]
 ====
@@ -312,26 +342,7 @@ Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop. See link:h
 [[hadoop.older.versions]]
 ==== Hadoop versions 0.20.x - 1.x
 
-HBase will lose data unless it is running on an HDFS that has a durable `sync` implementation.
-DO NOT use Hadoop 0.20.2, Hadoop 0.20.203.0, and Hadoop 0.20.204.0 which DO NOT have this attribute.
-Currently only Hadoop versions 0.20.205.x or any release in excess of this version -- this includes hadoop-1.0.0 -- have a working, durable sync.
-The Cloudera blog post link:http://www.cloudera.com/blog/2012/01/an-update-on-apache-hadoop-1-0/[An
-            update on Apache Hadoop 1.0] by Charles Zedlweski has a nice exposition on how all the Hadoop versions relate.
-It's worth checking out if you are having trouble making sense of the Hadoop version morass.
-
-Sync has to be explicitly enabled by setting `dfs.support.append` equal to true on both the client side -- in _hbase-site.xml_ -- and on the serverside in _hdfs-site.xml_ (The sync facility HBase needs is a subset of the append code path).
-
-[source,xml]
-----
-
-<property>
-  <name>dfs.support.append</name>
-  <value>true</value>
-</property>
-----
-
-You will have to restart your cluster after making this edit.
-Ignore the chicken-little comment you'll find in the _hdfs-default.xml_ in the description for the `dfs.support.append` configuration.
+DO NOT use Hadoop versions older than 2.2.0 for HBase versions greater than 1.0. Check release documentation if you are using an older version of HBase for Hadoop related information. 
 
 [[hadoop.security]]
 ==== Apache HBase on Secure Hadoop
@@ -373,8 +384,9 @@ See also <<casestudies.max.transfer.threads,casestudies.max.transfer.threads>> a
 === ZooKeeper Requirements
 
 ZooKeeper 3.4.x is required as of HBase 1.0.0.
-HBase makes use of the `multi` functionality that is only available since 3.4.0 (The `useMulti` configuration option defaults to `true` in HBase 1.0.0).
-See link:https://issues.apache.org/jira/browse/HBASE-12241[HBASE-12241 (The crash of regionServer when taking deadserver's replication queue breaks replication)] and link:https://issues.apache.org/jira/browse/HBASE-6775[HBASE-6775 (Use ZK.multi when available for HBASE-6710 0.92/0.94 compatibility fix)] for background.
+HBase makes use of the `multi` functionality that is only available since Zookeeper 3.4.0. The `hbase.zookeeper.useMulti` configuration property defaults to `true` in HBase 1.0.0.
+Refer to link:https://issues.apache.org/jira/browse/HBASE-12241[HBASE-12241 (The crash of regionServer when taking deadserver's replication queue breaks replication)] and link:https://issues.apache.org/jira/browse/HBASE-6775[HBASE-6775 (Use ZK.multi when available for HBASE-6710 0.92/0.94 compatibility fix)] for background.
+The property is deprecated and useMulti is always enabled in HBase 2.0.
 
 [[standalone_dist]]
 == HBase run modes: Standalone and Distributed
@@ -392,11 +404,12 @@ Set [var]+JAVA_HOME+ to point at the root of your +java+ install.
 This is the default mode.
 Standalone mode is what is described in the <<quickstart,quickstart>> section.
 In standalone mode, HBase does not use HDFS -- it uses the local filesystem instead -- and it runs all HBase daemons and a local ZooKeeper all up in the same JVM.
-Zookeeper binds to a well known port so clients may talk to HBase.
+ZooKeeper binds to a well known port so clients may talk to HBase.
 
+[[distributed]]
 === Distributed
 
-Distributed mode can be subdivided into distributed but all daemons run on a single node -- a.k.a _pseudo-distributed_ -- and _fully-distributed_ where the daemons are spread across all nodes in the cluster.
+Distributed mode can be subdivided into distributed but all daemons run on a single node -- a.k.a. _pseudo-distributed_ -- and _fully-distributed_ where the daemons are spread across all nodes in the cluster.
 The _pseudo-distributed_ vs. _fully-distributed_ nomenclature comes from Hadoop.
 
 Pseudo-distributed mode can run against the local filesystem or it can run against an instance of the _Hadoop Distributed File System_ (HDFS). Fully-distributed mode can ONLY run on HDFS.
@@ -433,7 +446,7 @@ In addition, the cluster is configured so that multiple cluster nodes enlist as
 These configuration basics are all demonstrated in <<quickstart_fully_distributed,quickstart-fully-distributed>>.
 
 .Distributed RegionServers
-Typically, your cluster will contain multiple RegionServers all running on different servers, as well as primary and backup Master and Zookeeper daemons.
+Typically, your cluster will contain multiple RegionServers all running on different servers, as well as primary and backup Master and ZooKeeper daemons.
 The _conf/regionservers_ file on the master server contains a list of hosts whose RegionServers are associated with this cluster.
 Each host is on a separate line.
 All hosts listed in this file will have their RegionServer processes started and stopped when the master server starts or stops.
@@ -526,7 +539,7 @@ HBase logs can be found in the _logs_ subdirectory.
 Check them out especially if HBase had trouble starting.
 
 HBase also puts up a UI listing vital attributes.
-By default it's deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational HTTP server at port 16030). If the Master is running on a host named `master.example.org` on the default port, point your browser at _http://master.example.org:16010_ to see the web interface.
+By default it's deployed on the Master host at port 16010 (HBase RegionServers listen on port 16020 by default and put up an informational HTTP server at port 16030). If the Master is running on a host named `master.example.org` on the default port, point your browser at pass:[http://master.example.org:16010] to see the web interface.
 
 Prior to HBase 0.98 the master UI was deployed on port 60010, and the HBase RegionServers UI on port 60030.
 
@@ -550,7 +563,7 @@ If you are running a distributed operation, be sure to wait until HBase has shut
 === _hbase-site.xml_ and _hbase-default.xml_
 
 Just as in Hadoop where you add site-specific HDFS configuration to the _hdfs-site.xml_ file, for HBase, site specific customizations go into the file _conf/hbase-site.xml_.
-For the list of configurable properties, see <<hbase_default_configurations,hbase default configurations>> below or view the raw _hbase-default.xml_ source file in the HBase source code at _src/main/resources_. 
+For the list of configurable properties, see <<hbase_default_configurations,hbase default configurations>> below or view the raw _hbase-default.xml_ source file in the HBase source code at _src/main/resources_.
 
 Not all configuration options make it out to _hbase-default.xml_.
 Configuration that it is thought rare anyone would change can exist only in code; the only way to turn up such configurations is via a reading of the source code itself.
@@ -558,7 +571,7 @@ Configuration that it is thought rare anyone would change can exist only in code
 Currently, changes here will require a cluster restart for HBase to notice the change.
 // hbase/src/main/asciidoc
 //
-include::../../../../target/asciidoc/hbase-default.adoc[]
+include::{docdir}/../../../target/asciidoc/hbase-default.adoc[]
 
 
 [[hbase.env.sh]]
@@ -590,7 +603,7 @@ ZooKeeper is where all these values are kept.
 Thus clients require the location of the ZooKeeper ensemble before they can do anything else.
 Usually this the ensemble location is kept out in the _hbase-site.xml_ and is picked up by the client from the `CLASSPATH`.
 
-If you are configuring an IDE to run a HBase client, you should include the _conf/_ directory on your classpath so _hbase-site.xml_ settings can be found (or add _src/test/resources_ to pick up the hbase-site.xml used by tests). 
+If you are configuring an IDE to run an HBase client, you should include the _conf/_ directory on your classpath so _hbase-site.xml_ settings can be found (or add _src/test/resources_ to pick up the hbase-site.xml used by tests).
 
 Minimally, a client of HBase needs several libraries in its `CLASSPATH` when connecting to a cluster, including:
 [source]
@@ -607,7 +620,7 @@ slf4j-log4j (slf4j-log4j12-1.5.8.jar)
 zookeeper (zookeeper-3.4.2.jar)
 ----
 
-An example basic _hbase-site.xml_ for client only might look as follows: 
+An example basic _hbase-site.xml_ for client only might look as follows:
 [source,xml]
 ----
 <?xml version="1.0"?>
@@ -683,8 +696,8 @@ Below we show what the main configuration files -- _hbase-site.xml_, _regionserv
     <name>hbase.cluster.distributed</name>
     <value>true</value>
     <description>The mode the cluster will be in. Possible values are
-      false: standalone and pseudo-distributed setups with managed Zookeeper
-      true: fully-distributed with unmanaged Zookeeper Quorum (see hbase-env.sh)
+      false: standalone and pseudo-distributed setups with managed ZooKeeper
+      true: fully-distributed with unmanaged ZooKeeper Quorum (see hbase-env.sh)
     </description>
   </property>
 </configuration>
@@ -716,7 +729,7 @@ The following lines in the _hbase-env.sh_ file show how to set the `JAVA_HOME` e
 
 ----
 # The java implementation to use.
-export JAVA_HOME=/usr/java/jdk1.7.0/
+export JAVA_HOME=/usr/java/jdk1.8.0/
 
 # The maximum amount of heap to use. Default is left to JVM default.
 export HBASE_HEAPSIZE=4G
@@ -744,14 +757,7 @@ See link:https://issues.apache.org/jira/browse/HBASE-6389[HBASE-6389 Modify the
             conditions to ensure that Master waits for sufficient number of Region Servers before
             starting region assignments] for more detail.
 
-[[backup.master.fail.fast]]
-==== If a backup Master exists, make the primary Master fail fast
-
-If the primary Master loses its connection with ZooKeeper, it will fall into a loop where it keeps trying to reconnect.
-Disable this functionality if you are running more than one Master: i.e. a backup Master.
-Failing to do so, the dying Master may continue to receive RPCs though another Master has assumed the role of primary.
-See the configuration <<fail.fast.expired.active.master,fail.fast.expired.active.master>>.
-
+[[recommended_configurations]]
 === Recommended Configurations
 
 [[recommended_configurations.zk]]
@@ -903,7 +909,7 @@ See <<master.processes.loadbalancer,master.processes.loadbalancer>> for more inf
 ==== Disabling Blockcache
 
 Do not turn off block cache (You'd do it by setting `hbase.block.cache.size` to zero). Currently we do not do well if you do this because the RegionServer will spend all its time loading HFile indices over and over again.
-If your working set it such that block cache does you no good, at least size the block cache such that HFile indices will stay up in the cache (you can get a rough idea on the size you need by surveying RegionServer UIs; you'll see index block size accounted near the top of the webpage).
+If your working set is such that block cache does you no good, at least size the block cache such that HFile indices will stay up in the cache (you can get a rough idea on the size you need by surveying RegionServer UIs; you'll see index block size accounted near the top of the webpage).
 
 [[nagles]]
 ==== link:http://en.wikipedia.org/wiki/Nagle's_algorithm[Nagle's] or the small package problem
@@ -916,7 +922,7 @@ You might also see the graphs on the tail of link:https://issues.apache.org/jira
 ==== Better Mean Time to Recover (MTTR)
 
 This section is about configurations that will make servers come back faster after a fail.
-See the Deveraj Das an Nicolas Liochon blog post link:http://hortonworks.com/blog/introduction-to-hbase-mean-time-to-recover-mttr/[Introduction to HBase Mean Time to Recover (MTTR)] for a brief introduction.
+See the Deveraj Das and Nicolas Liochon blog post link:http://hortonworks.com/blog/introduction-to-hbase-mean-time-to-recover-mttr/[Introduction to HBase Mean Time to Recover (MTTR)] for a brief introduction.
 
 The issue link:https://issues.apache.org/jira/browse/HBASE-8389[HBASE-8354 forces Namenode into loop with lease recovery requests] is messy but has a bunch of good discussion toward the end on low timeouts and how to effect faster recovery including citation of fixes added to HDFS. Read the Varun Sharma comments.
 The below suggested configurations are Varun's suggestions distilled and tested.
@@ -988,7 +994,7 @@ See the link:http://docs.oracle.com/javase/6/docs/technotes/guides/management/ag
 Historically, besides above port mentioned, JMX opens two additional random TCP listening ports, which could lead to port conflict problem. (See link:https://issues.apache.org/jira/browse/HBASE-10289[HBASE-10289] for details)
 
 As an alternative, You can use the coprocessor-based JMX implementation provided by HBase.
-To enable it in 0.99 or above, add below property in _hbase-site.xml_: 
+To enable it in 0.99 or above, add below property in _hbase-site.xml_:
 
 [source,xml]
 ----
@@ -1019,7 +1025,7 @@ The registry port can be shared with connector port in most cases, so you only n
 However if you want to use SSL communication, the 2 ports must be configured to different values.
 
 By default the password authentication and SSL communication is disabled.
-To enable password authentication, you need to update _hbase-env.sh_          like below: 
+To enable password authentication, you need to update _hbase-env.sh_          like below:
 [source,bash]
 ----
 export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.authenticate=true                  \
@@ -1046,7 +1052,7 @@ keytool -export -alias jconsole -keystore myKeyStore -file jconsole.cert
 keytool -import -alias jconsole -keystore jconsoleKeyStore -file jconsole.cert
 ----
 
-And then update _hbase-env.sh_ like below: 
+And then update _hbase-env.sh_ like below:
 
 [source,bash]
 ----
@@ -1068,12 +1074,12 @@ Finally start `jconsole` on the client using the key store:
 jconsole -J-Djavax.net.ssl.trustStore=/home/tianq/jconsoleKeyStore
 ----
 
-NOTE: To enable the HBase JMX implementation on Master, you also need to add below property in _hbase-site.xml_: 
+NOTE: To enable the HBase JMX implementation on Master, you also need to add below property in _hbase-site.xml_:
 
 [source,xml]
 ----
 <property>
-  <ame>hbase.coprocessor.master.classes</name>
+  <name>hbase.coprocessor.master.classes</name>
   <value>org.apache.hadoop.hbase.JMXListener</value>
 </property>
 ----

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index a99e903..1817dd3 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -27,203 +27,796 @@
 :icons: font
 :experimental:
 
-HBase coprocessors are modeled after the coprocessors which are part of Google's BigTable (http://www.scribd.com/doc/21631448/Dean-Keynote-Ladis2009, pages 66-67.). Coprocessors function in a similar way to Linux kernel modules.
-They provide a way to run server-level code against locally-stored data.
-The functionality they provide is very powerful, but also carries great risk and can have adverse effects on the system, at the level of the operating system.
-The information in this chapter is primarily sourced and heavily reused from Mingjie Lai's blog post at https://blogs.apache.org/hbase/entry/coprocessor_introduction.
+HBase Coprocessors are modeled after Google BigTable's coprocessor implementation
+(http://research.google.com/people/jeff/SOCC2010-keynote-slides.pdf pages 41-42.).
 
-Coprocessors are not designed to be used by end users of HBase, but by HBase developers who need to add specialized functionality to HBase.
-One example of the use of coprocessors is pluggable compaction and scan policies, which are provided as coprocessors in link:https://issues.apache.org/jira/browse/HBASE-6427[HBASE-6427].
+The coprocessor framework provides mechanisms for running your custom code directly on
+the RegionServers managing your data. Efforts are ongoing to bridge gaps between HBase's
+implementation and BigTable's architecture. For more information see
+link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047].
 
-== Coprocessor Framework
+The information in this chapter is primarily sourced and heavily reused from the following
+resources:
 
-The implementation of HBase coprocessors diverges from the BigTable implementation.
-The HBase framework provides a library and runtime environment for executing user code within the HBase region server and master processes.
+. Mingjie Lai's blog post
+link:https://blogs.apache.org/hbase/entry/coprocessor_introduction[Coprocessor Introduction].
+. Gaurav Bhardwaj's blog post
+link:http://www.3pillarglobal.com/insights/hbase-coprocessors[The How To Of HBase Coprocessors].
 
-The framework API is provided in the link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor] package.
+[WARNING]
+.Use Coprocessors At Your Own Risk
+====
+Coprocessors are an advanced feature of HBase and are intended to be used by system
+developers only. Because coprocessor code runs directly on the RegionServer and has
+direct access to your data, they introduce the risk of data corruption, man-in-the-middle
+attacks, or other malicious data access. Currently, there is no mechanism to prevent
+data corruption by coprocessors, though work is underway on
+link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047].
++
+In addition, there is no resource isolation, so a well-intentioned but misbehaving
+coprocessor can severely degrade cluster performance and stability.
+====
 
-Two different types of coprocessors are provided by the framework, based on their scope.
+== Coprocessor Overview
 
-.Types of Coprocessors
+In HBase, you fetch data using a `Get` or `Scan`, whereas in an RDBMS you use a SQL
+query. In order to fetch only the relevant data, you filter it using a HBase
+link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html[Filter]
+, whereas in an RDBMS you use a `WHERE` predicate.
 
-System Coprocessors::
-  System coprocessors are loaded globally on all tables and regions hosted by a region server.
+After fetching the data, you perform computations on it. This paradigm works well
+for "small data" with a few thousand rows and several columns. However, when you scale
+to billions of rows and millions of columns, moving large amounts of data across your
+network will create bottlenecks at the network layer, and the client needs to be powerful
+enough and have enough memory to handle the large amounts of data and the computations.
+In addition, the client code can grow large and complex.
+
+In this scenario, coprocessors might make sense. You can put the business computation
+code into a coprocessor which runs on the RegionServer, in the same location as the
+data, and returns the result to the client.
+
+This is only one scenario where using coprocessors can provide benefit. Following
+are some analogies which may help to explain some of the benefits of coprocessors.
+
+[[cp_analogies]]
+=== Coprocessor Analogies
+
+Triggers and Stored Procedure::
+  An Observer coprocessor is similar to a trigger in a RDBMS in that it executes
+  your code either before or after a specific event (such as a `Get` or `Put`)
+  occurs. An endpoint coprocessor is similar to a stored procedure in a RDBMS
+  because it allows you to perform custom computations on the data on the
+  RegionServer itself, rather than on the client.
 
-Table Coprocessors::
-  You can specify which coprocessors should be loaded on all regions for a table on a per-table basis.
+MapReduce::
+  MapReduce operates on the principle of moving the computation to the location of
+  the data. Coprocessors operate on the same principal.
 
-The framework provides two different aspects of extensions as well: _observers_ and _endpoints_.
+AOP::
+  If you are familiar with Aspect Oriented Programming (AOP), you can think of a coprocessor
+  as applying advice by intercepting a request and then running some custom code,
+  before passing the request on to its final destination (or even changing the destination).
+
+
+=== Coprocessor Implementation Overview
+
+. Either your class should extend one of the Coprocessor classes, such as
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver],
+or it should implement the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/Coprocessor.html[Coprocessor]
+or
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/CoprocessorService.html[CoprocessorService]
+interface.
+
+. Load the coprocessor, either statically (from the configuration) or dynamically,
+using HBase Shell. For more details see <<cp_loading,Loading Coprocessors>>.
+
+. Call the coprocessor from your client-side code. HBase handles the coprocessor
+transparently.
+
+The framework API is provided in the
+link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor]
+package.
+
+== Types of Coprocessors
+
+=== Observer Coprocessors
+
+Observer coprocessors are triggered either before or after a specific event occurs.
+Observers that happen before an event use methods that start with a `pre` prefix,
+such as link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#prePut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`prePut`]. Observers that happen just after an event override methods that start
+with a `post` prefix, such as link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#postPut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`postPut`].
+
+
+==== Use Cases for Observer Coprocessors
+Security::
+  Before performing a `Get` or `Put` operation, you can check for permission using
+  `preGet` or `prePut` methods.
+
+Referential Integrity::
+  HBase does not directly support the RDBMS concept of refential integrity, also known
+  as foreign keys. You can use a coprocessor to enforce such integrity. For instance,
+  if you have a business rule that every insert to the `users` table must be followed
+  by a corresponding entry in the `user_daily_attendance` table, you could implement
+  a coprocessor to use the `prePut` method on `user` to insert a record into `user_daily_attendance`.
+
+Secondary Indexes::
+  You can use a coprocessor to maintain secondary indexes. For more information, see
+  link:http://wiki.apache.org/hadoop/Hbase/SecondaryIndexing[SecondaryIndexing].
+
+
+==== Types of Observer Coprocessor
+
+RegionObserver::
+  A RegionObserver coprocessor allows you to observe events on a region, such as `Get`
+  and `Put` operations. See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver],
+  which implements the `RegionObserver` interface and will not break if new methods are added.
+
+RegionServerObserver::
+  A RegionServerObserver allows you to observe events related to the RegionServer's
+  operation, such as starting, stopping, or performing merges, commits, or rollbacks.
+  See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.html[RegionServerObserver].
+  Consider overriding the convenience class
+  https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.html[BaseMasterAndRegionObserver]
+  which implements both `MasterObserver` and `RegionServerObserver` interfaces and
+  will not break if new methods are added.
+
+MasterOvserver::
+  A MasterObserver allows you to observe events related to the HBase Master, such
+  as table creation, deletion, or schema modification. See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html[MasterObserver].
+  Consider overriding the convenience class
+  https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.html[BaseMasterAndRegionObserver],
+  which implements both `MasterObserver` and `RegionServerObserver` interfaces and
+  will not break if new methods are added.
+
+WalObserver::
+  A WalObserver allows you to observe events related to writes to the Write-Ahead
+  Log (WAL). See
+  link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/WALObserver.html[WALObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.html[BaseWALObserver],
+  which implements the `WalObserver` interface and will not break if new methods are added.
+
+<<cp_example,Examples>> provides working examples of observer coprocessors.
+
+
+
+[[cpeps]]
+=== Endpoint Coprocessor
+
+Endpoint processors allow you to perform computation at the location of the data.
+See <<cp_analogies, Coprocessor Analogy>>. An example is the need to calculate a running
+average or summation for an entire table which spans hundreds of regions.
+
+In contrast to observer coprocessors, where your code is run transparently, endpoint
+coprocessors must be explicitly invoked using the
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html#coprocessorService%28java.lang.Class,%20byte%5B%5D,%20byte%5B%5D,%20org.apache.hadoop.hbase.client.coprocessor.Batch.Call%29[CoprocessorService()]
+method available in
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html[Table],
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTableInterface.html[HTableInterface],
+or
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable].
+
+Starting with HBase 0.96, endpoint coprocessors are implemented using Google Protocol
+Buffers (protobuf). For more details on protobuf, see Google's
+link:https://developers.google.com/protocol-buffers/docs/proto[Protocol Buffer Guide].
+Endpoints Coprocessor written in version 0.94 are not compatible with version 0.96 or later.
+See
+link:https://issues.apache.org/jira/browse/HBASE-5448[HBASE-5448]). To upgrade your
+HBase cluster from 0.94 or earlier to 0.96 or later, you need to reimplement your
+coprocessor.
+
+Coprocessor Endpoints should make no use of HBase internals and
+only avail of public APIs; ideally a CPEP should depend on Interfaces
+and data structures only. This is not always possible but beware
+that doing so makes the Endpoint brittle, liable to breakage as HBase
+internals evolve. HBase internal APIs annotated as private or evolving
+do not have to respect semantic versioning rules or general java rules on
+deprecation before removal. While generated protobuf files are
+absent the hbase audience annotations -- they are created by the
+protobuf protoc tool which knows nothing of how HBase works --
+they should be consided `@InterfaceAudience.Private` so are liable to
+change.
+
+<<cp_example,Examples>> provides working examples of endpoint coprocessors.
+
+[[cp_loading]]
+== Loading Coprocessors
+
+To make your coprocessor available to HBase, it must be _loaded_, either statically
+(through the HBase configuration) or dynamically (using HBase Shell or the Java API).
+
+=== Static Loading
+
+Follow these steps to statically load your coprocessor. Keep in mind that you must
+restart HBase to unload a coprocessor that has been loaded statically.
+
+. Define the Coprocessor in _hbase-site.xml_, with a <property> element with a <name>
+and a <value> sub-element. The <name> should be one of the following:
++
+- `hbase.coprocessor.region.classes` for RegionObservers and Endpoints.
+- `hbase.coprocessor.wal.classes` for WALObservers.
+- `hbase.coprocessor.master.classes` for MasterObservers.
++
+<value> must contain the fully-qualified class name of your coprocessor's implementation
+class.
++
+For example to load a Coprocessor (implemented in class SumEndPoint.java) you have to create
+following entry in RegionServer's 'hbase-site.xml' file (generally located under 'conf' directory):
++
+[source,xml]
+----
+<property>
+    <name>hbase.coprocessor.region.classes</name>
+    <value>org.myname.hbase.coprocessor.endpoint.SumEndPoint</value>
+</property>
+----
++
+If multiple classes are specified for loading, the class names must be comma-separated.
+The framework attempts to load all the configured classes using the default class loader.
+Therefore, the jar file must reside on the server-side HBase classpath.
++
+Coprocessors which are loaded in this way will be active on all regions of all tables.
+These are also called system Coprocessor.
+The first listed Coprocessors will be assigned the priority `Coprocessor.Priority.SYSTEM`.
+Each subsequent coprocessor in the list will have its priority value incremented by one (which
+reduces its priority, because priorities have the natural sort order of Integers).
++
+When calling out to registered observers, the framework executes their callbacks methods in the
+sorted order of their priority. +
+Ties are broken arbitrarily.
 
-Observers::
-  Observers are analogous to triggers in conventional databases.
-  They allow you to insert user code by overriding upcall methods provided by the coprocessor framework.
-  Callback functions are executed from core HBase code when events occur.
-  Callbacks are handled by the framework, and the coprocessor itself only needs to insert the extended or alternate functionality.
+. Put your code on HBase's classpath. One easy way to do this is to drop the jar
+  (containing you code and all the dependencies) into the `lib/` directory in the
+  HBase installation.
 
-Endpoints (HBase 0.96.x and later)::
-  The implementation for endpoints changed significantly in HBase 0.96.x due to the introduction of protocol buffers (protobufs) (link:https://issues.apache.org/jira/browse/HBASE-5448[HBASE-5488]). If you created endpoints before 0.96.x, you will need to rewrite them.
-  Endpoints are now defined and callable as protobuf services, rather than endpoint invocations passed through as Writable blobs
+. Restart HBase.
 
-Endpoints (HBase 0.94.x and earlier)::
-  Dynamic RPC endpoints resemble stored procedures.
-  An endpoint can be invoked at any time from the client.
-  When it is invoked, it is executed remotely at the target region or regions, and results of the executions are returned to the client.
 
-== Examples
+=== Static Unloading
 
-An example of an observer is included in _hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java_.
-Several endpoint examples are included in the same directory.
+. Delete the coprocessor's <property> element, including sub-elements, from `hbase-site.xml`.
+. Restart HBase.
+. Optionally, remove the coprocessor's JAR file from the classpath or HBase's `lib/`
+  directory.
 
-== Building A Coprocessor
 
-Before you can build a processor, it must be developed, compiled, and packaged in a JAR file.
-The next step is to configure the coprocessor framework to use your coprocessor.
-You can load the coprocessor from your HBase configuration, so that the coprocessor starts with HBase, or you can configure the coprocessor from the HBase shell, as a table attribute, so that it is loaded dynamically when the table is opened or reopened.
+=== Dynamic Loading
 
-=== Load from Configuration
+You can also load a coprocessor dynamically, without restarting HBase. This may seem
+preferable to static loading, but dynamically loaded coprocessors are loaded on a
+per-table basis, and are only available to the table for which they were loaded. For
+this reason, dynamically loaded tables are sometimes called *Table Coprocessor*.
 
-To configure a coprocessor to be loaded when HBase starts, modify the RegionServer's _hbase-site.xml_ and configure one of the following properties, based on the type of observer you are configuring:
+In addition, dynamically loading a coprocessor acts as a schema change on the table,
+and the table must be taken offline to load the coprocessor.
 
-* `hbase.coprocessor.region.classes`for RegionObservers and Endpoints
-* `hbase.coprocessor.wal.classes`for WALObservers
-* `hbase.coprocessor.master.classes`for MasterObservers
+There are three ways to dynamically load Coprocessor.
 
-.Example RegionObserver Configuration
+[NOTE]
+.Assumptions
 ====
-In this example, one RegionObserver is configured for all the HBase tables.
+The below mentioned instructions makes the following assumptions:
 
-[source,xml]
+* A JAR called `coprocessor.jar` contains the Coprocessor implementation along with all of its
+dependencies.
+* The JAR is available in HDFS in some location like
+`hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar`.
+====
+
+[[load_coprocessor_in_shell]]
+==== Using HBase Shell
+
+. Disable the table using HBase Shell:
++
+[source]
 ----
-<property>
-  <name>hbase.coprocessor.region.classes</name>
-  <value>org.apache.hadoop.hbase.coprocessor.AggregateImplementation</value>
-</property>
+hbase> disable 'users'
 ----
-====
 
-If multiple classes are specified for loading, the class names must be comma-separated.
-The framework attempts to load all the configured classes using the default class loader.
-Therefore, the jar file must reside on the server-side HBase classpath.
+. Load the Coprocessor, using a command like the following:
++
+[source]
+----
+hbase alter 'users', METHOD => 'table_att', 'Coprocessor'=>'hdfs://<namenode>:<port>/
+user/<hadoop-user>/coprocessor.jar| org.myname.hbase.Coprocessor.RegionObserverExample|1073741823|
+arg1=1,arg2=2'
+----
++
+The Coprocessor framework will try to read the class information from the coprocessor table
+attribute value.
+The value contains four pieces of information which are separated by the pipe (`|`) character.
++
+* File path: The jar file containing the Coprocessor implementation must be in a location where
+all region servers can read it. +
+You could copy the file onto the local disk on each region server, but it is recommended to store
+it in HDFS. +
+https://issues.apache.org/jira/browse/HBASE-14548[HBASE-14548] allows a directory containing the jars
+or some wildcards to be specified, such as: hdfs://<namenode>:<port>/user/<hadoop-user>/ or
+hdfs://<namenode>:<port>/user/<hadoop-user>/*.jar. Please note that if a directory is specified,
+all jar files(.jar) in the directory are added. It does not search for files in sub-directories.
+Do not use a wildcard if you would like to specify a directory. This enhancement applies to the
+usage via the JAVA API as well.
+* Class name: The full class name of the Coprocessor.
+* Priority: An integer. The framework will determine the execution sequence of all configured
+observers registered at the same hook using priorities. This field can be left blank. In that
+case the framework will assign a default priority value.
+* Arguments (Optional): This field is passed to the Coprocessor implementation. This is optional.
+
+. Enable the table.
++
+----
+hbase(main):003:0> enable 'users'
+----
 
-Coprocessors which are loaded in this way will be active on all regions of all tables.
-These are the system coprocessor introduced earlier.
-The first listed coprocessors will be assigned the priority `Coprocessor.Priority.SYSTEM`.
-Each subsequent coprocessor in the list will have its priority value incremented by one (which reduces its priority, because priorities have the natural sort order of Integers).
+. Verify that the coprocessor loaded:
++
+----
+hbase(main):04:0> describe 'users'
+----
++
+The coprocessor should be listed in the `TABLE_ATTRIBUTES`.
 
-When calling out to registered observers, the framework executes their callbacks methods in the sorted order of their priority.
-Ties are broken arbitrarily.
+==== Using the Java API (all HBase versions)
 
-=== Load from the HBase Shell
+The following Java code shows how to use the `setValue()` method of `HTableDescriptor`
+to load a coprocessor on the `users` table.
 
-You can load a coprocessor on a specific table via a table attribute.
-The following example will load the `FooRegionObserver` observer when table `t1` is read or re-read.
+[source,java]
+----
+TableName tableName = TableName.valueOf("users");
+String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
+Configuration conf = HBaseConfiguration.create();
+Connection connection = ConnectionFactory.createConnection(conf);
+Admin admin = connection.getAdmin();
+admin.disableTable(tableName);
+HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
+HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
+columnFamily1.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily1);
+HColumnDescriptor columnFamily2 = new HColumnDescriptor("salaryDet");
+columnFamily2.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily2);
+hTableDescriptor.setValue("COPROCESSOR$1", path + "|"
++ RegionObserverExample.class.getCanonicalName() + "|"
++ Coprocessor.PRIORITY_USER);
+admin.modifyTable(tableName, hTableDescriptor);
+admin.enableTable(tableName);
+----
 
-.Load a Coprocessor On a Table Using HBase Shell
-====
+==== Using the Java API (HBase 0.96+ only)
+
+In HBase 0.96 and newer, the `addCoprocessor()` method of `HTableDescriptor` provides
+an easier way to load a coprocessor dynamically.
+
+[source,java]
 ----
-hbase(main):005:0>  alter 't1', METHOD => 'table_att',
-  'coprocessor'=>'hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2'
-Updating all regions with the new schema...
-1/1 regions updated.
-Done.
-0 row(s) in 1.0730 seconds
-
-hbase(main):006:0> describe 't1'
-DESCRIPTION                                                        ENABLED
- {NAME => 't1', coprocessor$1 => 'hdfs:///foo.jar|com.foo.FooRegio false
- nObserver|1001|arg1=1,arg2=2', FAMILIES => [{NAME => 'c1', DATA_B
- LOCK_ENCODING => 'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE
-  => '0', VERSIONS => '3', COMPRESSION => 'NONE', MIN_VERSIONS =>
- '0', TTL => '2147483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZ
- E => '65536', IN_MEMORY => 'false', ENCODE_ON_DISK => 'true', BLO
- CKCACHE => 'true'}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE',
-  BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3'
- , COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '2147483647'
- , KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY
- => 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
-1 row(s) in 0.0190 seconds
+TableName tableName = TableName.valueOf("users");
+Path path = new Path("hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar");
+Configuration conf = HBaseConfiguration.create();
+Connection connection = ConnectionFactory.createConnection(conf);
+Admin admin = connection.getAdmin();
+admin.disableTable(tableName);
+HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
+HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
+columnFamily1.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily1);
+HColumnDescriptor columnFamily2 = new HColumnDescriptor("salaryDet");
+columnFamily2.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily2);
+hTableDescriptor.addCoprocessor(RegionObserverExample.class.getCanonicalName(), path,
+Coprocessor.PRIORITY_USER, null);
+admin.modifyTable(tableName, hTableDescriptor);
+admin.enableTable(tableName);
 ----
-====
 
-The coprocessor framework will try to read the class information from the coprocessor table attribute value.
-The value contains four pieces of information which are separated by the `|` character.
+WARNING: There is no guarantee that the framework will load a given Coprocessor successfully.
+For example, the shell command neither guarantees a jar file exists at a particular location nor
+verifies whether the given class is actually contained in the jar file.
 
-* File path: The jar file containing the coprocessor implementation must be in a location where all region servers can read it.
-  You could copy the file onto the local disk on each region server, but it is recommended to store it in HDFS.
-* Class name: The full class name of the coprocessor.
-* Priority: An integer.
-  The framework will determine the execution sequence of all configured observers registered at the same hook using priorities.
-  This field can be left blank.
-  In that case the framework will assign a default priority value.
-* Arguments: This field is passed to the coprocessor implementation.
 
-.Unload a Coprocessor From a Table Using HBase Shell
-====
+=== Dynamic Unloading
+
+==== Using HBase Shell
+
+. Disable the table.
++
+[source]
+----
+hbase> disable 'users'
 ----
 
-hbase(main):007:0> alter 't1', METHOD => 'table_att_unset',
-hbase(main):008:0*   NAME => 'coprocessor$1'
-Updating all regions with the new schema...
-1/1 regions updated.
-Done.
-0 row(s) in 1.1130 seconds
-
-hbase(main):009:0> describe 't1'
-DESCRIPTION                                                        ENABLED
- {NAME => 't1', FAMILIES => [{NAME => 'c1', DATA_BLOCK_ENCODING => false
-  'NONE', BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSION
- S => '3', COMPRESSION => 'NONE', MIN_VERSIONS => '0', TTL => '214
- 7483647', KEEP_DELETED_CELLS => 'false', BLOCKSIZE => '65536', IN
- _MEMORY => 'false', ENCODE_ON_DISK => 'true', BLOCKCACHE => 'true
- '}, {NAME => 'f1', DATA_BLOCK_ENCODING => 'NONE', BLOOMFILTER =>
- 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION =>
-  'NONE', MIN_VERSIONS => '0', TTL => '2147483647', KEEP_DELETED_C
- ELLS => 'false', BLOCKSIZE => '65536', IN_MEMORY => 'false', ENCO
- DE_ON_DISK => 'true', BLOCKCACHE => 'true'}]}
-1 row(s) in 0.0180 seconds
+. Alter the table to remove the coprocessor.
++
+[source]
 ----
-====
+hbase> alter 'users', METHOD => 'table_att_unset', NAME => 'coprocessor$1'
+----
+
+. Enable the table.
++
+[source]
+----
+hbase> enable 'users'
+----
+
+==== Using the Java API
+
+Reload the table definition without setting the value of the coprocessor either by
+using `setValue()` or `addCoprocessor()` methods. This will remove any coprocessor
+attached to the table.
+
+[source,java]
+----
+TableName tableName = TableName.valueOf("users");
+String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
+Configuration conf = HBaseConfiguration.create();
+Connection connection = ConnectionFactory.createConnection(conf);
+Admin admin = connection.getAdmin();
+admin.disableTable(tableName);
+HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
+HColumnDescriptor columnFamily1 = new HColumnDescriptor("personalDet");
+columnFamily1.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily1);
+HColumnDescriptor columnFamily2 = new HColumnDescriptor("salaryDet");
+columnFamily2.setMaxVersions(3);
+hTableDescriptor.addFamily(columnFamily2);
+admin.modifyTable(tableName, hTableDescriptor);
+admin.enableTable(tableName);
+----
+
+In HBase 0.96 and newer, you can instead use the `removeCoprocessor()` method of the
+`HTableDescriptor` class.
+
+
+[[cp_example]]
+== Examples
+HBase ships examples for Observer Coprocessor in
+link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.html[ZooKeeperScanPolicyObserver]
+and for Endpoint Coprocessor in
+link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.html[RowCountEndpoint]
+
+A more detailed example is given below.
+
+These examples assume a table called `users`, which has two column families `personalDet`
+and `salaryDet`, containing personal and salary details. Below is the graphical representation
+of the `users` table.
+
+.Users Table
+[width="100%",cols="7",options="header,footer"]
+|====================
+| 3+|personalDet  3+|salaryDet
+|*rowkey* |*name* |*lastname* |*dob* |*gross* |*net* |*allowances*
+|admin |Admin |Admin |  3+|
+|cdickens |Charles |Dickens |02/07/1812 |10000 |8000 |2000
+|jverne |Jules |Verne |02/08/1828 |12000 |9000 |3000
+|====================
+
+
+=== Observer Example
+
+The following Observer coprocessor prevents the details of the user `admin` from being
+returned in a `Get` or `Scan` of the `users` table.
+
+. Write a class that extends the
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver]
+class.
 
-WARNING: There is no guarantee that the framework will load a given coprocessor successfully.
-For example, the shell command neither guarantees a jar file exists at a particular location nor verifies whether the given class is actually contained in the jar file.
+. Override the `preGetOp()` method (the `preGet()` method is deprecated) to check
+whether the client has queried for the rowkey with value `admin`. If so, return an
+empty result. Otherwise, process the request as normal.
 
-== Check the Status of a Coprocessor
+. Put your code and dependencies in a JAR file.
 
-To check the status of a coprocessor after it has been configured, use the `status` HBase Shell command.
+. Place the JAR in HDFS where HBase can locate it.
 
+. Load the Coprocessor.
+
+. Write a simple program to test it.
+
+Following are the implementation of the above steps:
+
+
+[source,java]
+----
+public class RegionObserverExample extends BaseRegionObserver {
+
+    private static final byte[] ADMIN = Bytes.toBytes("admin");
+    private static final byte[] COLUMN_FAMILY = Bytes.toBytes("details");
+    private static final byte[] COLUMN = Bytes.toBytes("Admin_det");
+    private static final byte[] VALUE = Bytes.toBytes("You can't see Admin details");
+
+    @Override
+    public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e, final Get get, final List<Cell> results)
+    throws IOException {
+
+        if (Bytes.equals(get.getRow(),ADMIN)) {
+            Cell c = CellUtil.createCell(get.getRow(),COLUMN_FAMILY, COLUMN,
+            System.currentTimeMillis(), (byte)4, VALUE);
+            results.add(c);
+            e.bypass();
+        }
+    }
+}
+----
+
+Overriding the `preGetOp()` will only work for `Get` operations. You also need to override
+the `preScannerOpen()` method to filter the `admin` row from scan results.
+
+[source,java]
+----
+@Override
+public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e, final Scan scan,
+final RegionScanner s) throws IOException {
+
+    Filter filter = new RowFilter(CompareOp.NOT_EQUAL, new BinaryComparator(ADMIN));
+    scan.setFilter(filter);
+    return s;
+}
 ----
 
-hbase(main):020:0> status 'detailed'
-version 0.92-tm-6
-0 regionsInTransition
-master coprocessors: []
-1 live servers
-    localhost:52761 1328082515520
-        requestsPerSecond=3, numberOfOnlineRegions=3, usedHeapMB=32, maxHeapMB=995
-        -ROOT-,,0
-            numberOfStores=1, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
-storefileIndexSizeMB=0, readRequestsCount=54, writeRequestsCount=1, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
-totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN, coprocessors=[]
-        .META.,,1
-            numberOfStores=1, numberOfStorefiles=0, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
-storefileIndexSizeMB=0, readRequestsCount=97, writeRequestsCount=4, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
-totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN, coprocessors=[]
-        t1,,1328082575190.c0491168a27620ffe653ec6c04c9b4d1.
-            numberOfStores=2, numberOfStorefiles=1, storefileUncompressedSizeMB=0, storefileSizeMB=0, memstoreSizeMB=0,
-storefileIndexSizeMB=0, readRequestsCount=0, writeRequestsCount=0, rootIndexSizeKB=0, totalStaticIndexSizeKB=0,
-totalStaticBloomSizeKB=0, totalCompactingKVs=0, currentCompactedKVs=0, compactionProgressPct=NaN,
-coprocessors=[AggregateImplementation]
-0 dead servers
+This method works but there is a _side effect_. If the client has used a filter in
+its scan, that filter will be replaced by this filter. Instead, you can explicitly
+remove any `admin` results from the scan:
+
+[source,java]
+----
+@Override
+public boolean postScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e, final InternalScanner s,
+final List<Result> results, final int limit, final boolean hasMore) throws IOException {
+	Result result = null;
+    Iterator<Result> iterator = results.iterator();
+    while (iterator.hasNext()) {
+    result = iterator.next();
+        if (Bytes.equals(result.getRow(), ROWKEY)) {
+            iterator.remove();
+            break;
+        }
+    }
+    return hasMore;
+}
 ----
 
+=== Endpoint Example
+
+Still using the `users` table, this example implements a coprocessor to calculate
+the sum of all employee salaries, using an endpoint coprocessor.
+
+. Create a '.proto' file defining your service.
++
+[source]
+----
+option java_package = "org.myname.hbase.coprocessor.autogenerated";
+option java_outer_classname = "Sum";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+message SumRequest {
+    required string family = 1;
+    required string column = 2;
+}
+
+message SumResponse {
+  required int64 sum = 1 [default = 0];
+}
+
+service SumService {
+  rpc getSum(SumRequest)
+    returns (SumResponse);
+}
+----
+
+. Execute the `protoc` command to generate the Java code from the above .proto' file.
++
+[source]
+----
+$ mkdir src
+$ protoc --java_out=src ./sum.proto
+----
++
+This will generate a class call `Sum.java`.
+
+. Write a class that extends the generated service class, implement the `Coprocessor`
+and `CoprocessorService` classes, and override the service method.
++
+WARNING: If you load a coprocessor from `hbase-site.xml` and then load the same coprocessor
+again using HBase Shell, it will be loaded a second time. The same class will
+exist twice, and the second instance will have a higher ID (and thus a lower priority).
+The effect is that the duplicate coprocessor is effectively ignored.
++
+[source, java]
+----
+public class SumEndPoint extends SumService implements Coprocessor, CoprocessorService {
+
+    private RegionCoprocessorEnvironment env;
+
+    @Override
+    public Service getService() {
+        return this;
+    }
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+        if (env instanceof RegionCoprocessorEnvironment) {
+            this.env = (RegionCoprocessorEnvironment)env;
+        } else {
+            throw new CoprocessorException("Must be loaded on a table region!");
+        }
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+        // do mothing
+    }
+
+    @Override
+    public void getSum(RpcController controller, SumRequest request, RpcCallback done) {
+        Scan scan = new Scan();
+        scan.addFamily(Bytes.toBytes(request.getFamily()));
+        scan.addColumn(Bytes.toBytes(request.getFamily()), Bytes.toBytes(request.getColumn()));
+        SumResponse response = null;
+        InternalScanner scanner = null;
+        try {
+            scanner = env.getRegion().getScanner(scan);
+            List results = new ArrayList();
+            boolean hasMore = false;
+                        long sum = 0L;
+                do {
+                        hasMore = scanner.next(results);
+                        for (Cell cell : results) {
+                            sum = sum + Bytes.toLong(CellUtil.cloneValue(cell));
+                     }
+                        results.clear();
+                } while (hasMore);
+
+                response = SumResponse.newBuilder().setSum(sum).build();
+
+        } catch (IOException ioe) {
+            ResponseConverter.setControllerException(controller, ioe);
+        } finally {
+            if (scanner != null) {
+                try {
+                    scanner.close();
+                } catch (IOException ignored) {}
+            }
+        }
+        done.run(response);
+    }
+}
+----
++
+[source, java]
+----
+Configuration conf = HBaseConfiguration.create();
+// Use below code for HBase version 1.x.x or above.
+Connection connection = ConnectionFactory.createConnection(conf);
+TableName tableName = TableName.valueOf("users");
+Table table = connection.getTable(tableName);
+
+//Use below code HBase version 0.98.xx or below.
+//HConnection connection = HConnectionManager.createConnection(conf);
+//HTableInterface table = connection.getTable("users");
+
+final SumRequest request = SumRequest.newBuilder().setFamily("salaryDet").setColumn("gross")
+                            .build();
+try {
+Map<byte[], Long> results = table.CoprocessorService (SumService.class, null, null,
+new Batch.Call<SumService, Long>() {
+    @Override
+        public Long call(SumService aggregate) throws IOException {
+BlockingRpcCallback rpcCallback = new BlockingRpcCallback();
+            aggregate.getSum(null, request, rpcCallback);
+            SumResponse response = rpcCallback.get();
+            return response.hasSum() ? response.getSum() : 0L;
+        }
+    });
+    for (Long sum : results.values()) {
+        System.out.println("Sum = " + sum);
+    }
+} catch (ServiceException e) {
+e.printStackTrace();
+} catch (Throwable e) {
+    e.printStackTrace();
+}
+----
+
+. Load the Coprocessor.
+
+. Write a client code to call the Coprocessor.
+
+
+== Guidelines For Deploying A Coprocessor
+
+Bundling Coprocessors::
+  You can bundle all classes for a coprocessor into a
+  single JAR on the RegionServer's classpath, for easy deployment. Otherwise,
+  place all dependencies  on the RegionServer's classpath so that they can be
+  loaded during RegionServer start-up.  The classpath for a RegionServer is set
+  in the RegionServer's `hbase-env.sh` file.
+Automating Deployment::
+  You can use a tool such as Puppet, Chef, or
+  Ansible to ship the JAR for the coprocessor  to the required location on your
+  RegionServers' filesystems and restart each RegionServer,  to automate
+  coprocessor deployment. Details for such set-ups are out of scope of  this
+  document.
+Updating a Coprocessor::
+  Deploying a new version of a given coprocessor is not as simple as disabling it,
+  replacing the JAR, and re-enabling the coprocessor. This is because you cannot
+  reload a class in a JVM unless you delete all the current references to it.
+  Since the current JVM has reference to the existing coprocessor, you must restart
+  the JVM, by restarting the RegionServer, in order to replace it. This behavior
+  is not expected to change.
+Coprocessor Logging::
+  The Coprocessor framework does not provide an API for logging beyond standard Java
+  logging.
+Coprocessor Configuration::
+  If you do not want to load coprocessors from the HBase Shell, you can add their configuration
+  properties to `hbase-site.xml`. In <<load_coprocessor_in_shell>>, two arguments are
+  set: `arg1=1,arg2=2`. These could have been added to `hbase-site.xml` as follows:
+[source,xml]
+----
+<property>
+  <name>arg1</name>
+  <value>1</value>
+</property>
+<property>
+  <name>arg2</name>
+  <value>2</value>
+</property>
+----
+Then you can read the configuration using code like the following:
+[source,java]
+----
+Configuration conf = HBaseConfiguration.create();
+// Use below code for HBase version 1.x.x or above.
+Connection connection = ConnectionFactory.createConnection(conf);
+TableName tableName = TableName.valueOf("users");
+Table table = connection.getTable(tableName);
+
+//Use below code HBase version 0.98.xx or below.
+//HConnection connection = HConnectionManager.createConnection(conf);
+//HTableInterface table = connection.getTable("users");
+
+Get get = new Get(Bytes.toBytes("admin"));
+Result result = table.get(get);
+for (Cell c : result.rawCells()) {
+    System.out.println(Bytes.toString(CellUtil.cloneRow(c))
+        + "==> " + Bytes.toString(CellUtil.cloneFamily(c))
+        + "{" + Bytes.toString(CellUtil.cloneQualifier(c))
+        + ":" + Bytes.toLong(CellUtil.cloneValue(c)) + "}");
+}
+Scan scan = new Scan();
+ResultScanner scanner = table.getScanner(scan);
+for (Result res : scanner) {
+    for (Cell c : res.rawCells()) {
+        System.out.println(Bytes.toString(CellUtil.cloneRow(c))
+        + " ==> " + Bytes.toString(CellUtil.cloneFamily(c))
+        + " {" + Bytes.toString(CellUtil.cloneQualifier(c))
+        + ":" + Bytes.toLong(CellUtil.cloneValue(c))
+        + "}");
+    }
+}
+----
+
+
+
+
 == Monitor Time Spent in Coprocessors
 
-HBase 0.98.5 introduced the ability to monitor some statistics relating to the amount of time spent executing a given coprocessor.
-You can see these statistics via the HBase Metrics framework (see <<hbase_metrics>> or the Web UI for a given Region Server, via the _Coprocessor Metrics_ tab.
-These statistics are valuable for debugging and benchmarking the performance impact of a given coprocessor on your cluster.
+HBase 0.98.5 introduced the ability to monitor some statistics relating to the amount of time
+spent executing a given Coprocessor.
+You can see these statistics via the HBase Metrics framework (see <<hbase_metrics>> or the Web UI
+for a given Region Server, via the _Coprocessor Metrics_ tab.
+These statistics are valuable for debugging and benchmarking the performance impact of a given
+Coprocessor on your cluster.
 Tracked statistics include min, max, average, and 90th, 95th, and 99th percentile.
 All times are shown in milliseconds.
-The statistics are calculated over coprocessor execution samples recorded during the reporting interval, which is 10 seconds by default.
+The statistics are calculated over Coprocessor execution samples recorded during the reporting
+interval, which is 10 seconds by default.
 The metrics sampling rate as described in <<hbase_metrics>>.
 
 .Coprocessor Metrics UI

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cb8a436/src/main/asciidoc/_chapters/datamodel.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/datamodel.adoc b/src/main/asciidoc/_chapters/datamodel.adoc
index b76adc8..30465fb 100644
--- a/src/main/asciidoc/_chapters/datamodel.adoc
+++ b/src/main/asciidoc/_chapters/datamodel.adoc
@@ -93,7 +93,7 @@ The colon character (`:`) delimits the column family from the column family _qua
 |===
 |Row Key |Time Stamp  |ColumnFamily `contents` |ColumnFamily `anchor`|ColumnFamily `people`
 |"com.cnn.www" |t9    | |anchor:cnnsi.com = "CNN"   |
-|"com.cnn.www" |t8    | |anchor:my.look.ca = "CNN.com" |  
+|"com.cnn.www" |t8    | |anchor:my.look.ca = "CNN.com" |
 |"com.cnn.www" |t6  | contents:html = "<html>..."    | |
 |"com.cnn.www" |t5  | contents:html = "<html>..."    | |
 |"com.cnn.www" |t3  | contents:html = "<html>..."    | |
@@ -171,7 +171,7 @@ For more information about the internals of how Apache HBase stores data, see <<
 A namespace is a logical grouping of tables analogous to a database in relation database systems.
 This abstraction lays the groundwork for upcoming multi-tenancy related features:
 
-* Quota Management (link:https://issues.apache.org/jira/browse/HBASE-8410[HBASE-8410]) - Restrict the amount of resources (ie regions, tables) a namespace can consume.
+* Quota Management (link:https://issues.apache.org/jira/browse/HBASE-8410[HBASE-8410]) - Restrict the amount of resources (i.e. regions, tables) a namespace can consume.
 * Namespace Security Administration (link:https://issues.apache.org/jira/browse/HBASE-9206[HBASE-9206]) - Provide another level of security administration for tenants.
 * Region server groups (link:https://issues.apache.org/jira/browse/HBASE-6721[HBASE-6721]) - A namespace/table can be pinned onto a subset of RegionServers thus guaranteeing a course level of isolation.
 
@@ -257,7 +257,7 @@ For example, the columns _courses:history_ and _courses:math_ are both members o
 The colon character (`:`) delimits the column family from the column family qualifier.
 The column family prefix must be composed of _printable_ characters.
 The qualifying tail, the column family _qualifier_, can be made of any arbitrary bytes.
-Column families must be declared up front at schema definition time whereas columns do not need to be defined at schema time but can be conjured on the fly while the table is up an running.
+Column families must be declared up front at schema definition time whereas columns do not need to be defined at schema time but can be conjured on the fly while the table is up and running.
 
 Physically, all column family members are stored together on the filesystem.
 Because tunings and storage specifications are done at the column family level, it is advised that all column family members have the same general access pattern and size characteristics.
@@ -279,7 +279,7 @@ Gets are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hba
 
 === Put
 
-link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#put(org.apache.hadoop.hbase.client.Put)[Table.put] (writeBuffer) or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch(java.util.List, java.lang.Object[])[Table.batch] (non-writeBuffer).
+link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Put.html[Put] either adds new rows to a table (if the key is new) or can update existing rows (if the key already exists). Puts are executed via link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#put(org.apache.hadoop.hbase.client.Put)[Table.put] (writeBuffer) or link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Table.html#batch(java.util.List,%20java.lang.Object%5B%5D)[Table.batch] (non-writeBuffer).
 
 [[scan]]
 === Scans
@@ -542,6 +542,7 @@ Thus, while HBase can support not only a wide number of columns per row, but a h
 The only way to get a complete set of columns that exist for a ColumnFamily is to process all the rows.
 For more information about how HBase stores data internally, see <<keyvalue,keyvalue>>.
 
+[[joins]]
 == Joins
 
 Whether HBase supports joins is a common question on the dist-list, and there is a simple answer:  it doesn't, at not least in the way that RDBMS' support them (e.g., with equi-joins or outer-joins in SQL).  As has been illustrated in this chapter, the read data model operations in HBase are Get and Scan.
@@ -552,7 +553,7 @@ hash-joins). So which is the best approach? It depends on what you are trying to
 
 == ACID
 
-See link:http://hbase.apache.org/acid-semantics.html[ACID Semantics].
+See link:/acid-semantics.html[ACID Semantics].
 Lars Hofhansl has also written a note on link:http://hadoop-hbase.blogspot.com/2012/03/acid-in-hbase.html[ACID in HBase].
 
 ifdef::backend-docbook[]