hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ndimi...@apache.org
Subject [1/4] hbase git commit: updating docs from master
Date Sun, 17 Jan 2016 00:07:15 GMT
Repository: hbase
Updated Branches:
  refs/heads/branch-1.1 5a1cfc1e9 -> c07ddc6db


http://git-wip-us.apache.org/repos/asf/hbase/blob/c07ddc6d/src/main/asciidoc/_chapters/rpc.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/rpc.adoc b/src/main/asciidoc/_chapters/rpc.adoc
index c319f39..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
 
@@ -79,7 +79,7 @@ link:https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=blob;f=hbase-protocol
 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]
@@ -106,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
 
@@ -118,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
 
@@ -182,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.
@@ -191,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.
@@ -205,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/c07ddc6d/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 a212a5c..5cf8d12 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -53,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);
 
@@ -76,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 <<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
 
@@ -187,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.
@@ -339,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.
@@ -406,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.
 
@@ -514,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]
 ----
@@ -605,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.
@@ -632,8 +676,13 @@ For more information, see <<coprocessors,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
@@ -733,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:
 
@@ -751,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...
 
@@ -796,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.
@@ -854,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:
@@ -879,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
@@ -902,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:
 
@@ -927,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
@@ -941,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).
 
@@ -1040,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.

http://git-wip-us.apache.org/repos/asf/hbase/blob/c07ddc6d/src/main/asciidoc/_chapters/security.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/security.adoc b/src/main/asciidoc/_chapters/security.adoc
index fb2a6b0..c346435 100644
--- a/src/main/asciidoc/_chapters/security.adoc
+++ b/src/main/asciidoc/_chapters/security.adoc
@@ -236,39 +236,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
 
@@ -306,6 +308,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>
@@ -331,7 +337,7 @@ To enable REST gateway Kerberos authentication for client access, add the follow
 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]]
@@ -343,7 +349,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.
@@ -1117,7 +1123,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.
@@ -1332,11 +1338,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.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.
+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
 ====
@@ -1582,8 +1598,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/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 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/c07ddc6d/src/main/asciidoc/_chapters/shell.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/shell.adoc b/src/main/asciidoc/_chapters/shell.adoc
index 237089e..a4237fd 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.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c07ddc6d/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..37503e9
--- /dev/null
+++ b/src/main/asciidoc/_chapters/spark.adoc
@@ -0,0 +1,451 @@
+////
+/**
+ *
+ * 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’s 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
+
+Spark bulk load follows very closely to the MapReduce implementation of bulk
+load. In short, a partitioner partitions based on region splits and
+the row keys are sent to the reducers in order, so that HFiles can be written
+out. In Spark terms, the bulk load will be focused around a
+`repartitionAndSortWithinPartitions` followed by a `foreachPartition`.
+
+The only major difference with the Spark implementation compared to the
+MapReduce implementation is that the column qualifier is included in the shuffle
+ordering process. This was done because the MapReduce bulk load implementation
+would have memory issues with loading rows with a large numbers of columns, as a
+result of the sorting of those columns being done in the memory of the reducer JVM.
+Instead, that ordering is done in the Spark Shuffle, so there should no longer
+be a limit to the number of columns in a row for bulk loading.
+
+.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 = ...
+
+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 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)))
+----
+====
+
+== SparkSQL/DataFrames
+
+http://spark.apache.org/sql/[SparkSQL] is a subproject of Spark that supports
+SQL that will compute down to a Spark DAG. In addition,SparkSQL is a heavy user
+of DataFrames. DataFrames are like RDDs with schema information.
+
+The HBase-Spark module includes support for Spark SQL and DataFrames, which allows
+you to write SparkSQL directly on HBase tables. In addition the HBase-Spark
+will push down query filtering logic to HBase.
+
+=== Predicate Push Down
+
+There are two examples of predicate push down in the HBase-Spark implementation.
+The first example shows the push down of filtering logic on the RowKey. HBase-Spark
+will reduce the filters on RowKeys down to a set of Get and/or Scan commands.
+
+NOTE: The Scans are distributed scans, rather than a single client scan operation.
+
+If the query looks something like the following, the logic will push down and get
+the rows through 3 Gets and 0 Scans. We can do gets because all the operations
+are `equal` operations.
+
+[source,sql]
+----
+SELECT
+  KEY_FIELD,
+  B_FIELD,
+  A_FIELD
+FROM hbaseTmp
+WHERE (KEY_FIELD = 'get1' or KEY_FIELD = 'get2' or KEY_FIELD = 'get3')
+----
+
+Now let's look at an example where we will end up doing two scans on HBase.
+
+[source, sql]
+----
+SELECT
+  KEY_FIELD,
+  B_FIELD,
+  A_FIELD
+FROM hbaseTmp
+WHERE KEY_FIELD < 'get2' or KEY_FIELD > 'get3'
+----
+
+In this example we will get 0 Gets and 2 Scans. One scan will load everything
+from the first row in the table until “get2” and the second scan will get
+everything from “get3” until the last row in the table.
+
+The next query is a good example of having a good deal of range checks. However
+the ranges overlap. To the code will be smart enough to get the following data
+in a single scan that encompasses all the data asked by the query.
+
+[source, sql]
+----
+SELECT
+  KEY_FIELD,
+  B_FIELD,
+  A_FIELD
+FROM hbaseTmp
+WHERE
+  (KEY_FIELD >= 'get1' and KEY_FIELD <= 'get3') or
+  (KEY_FIELD > 'get3' and KEY_FIELD <= 'get5')
+----
+
+The second example of push down functionality offered by the HBase-Spark module
+is the ability to push down filter logic for column and cell fields. Just like
+the RowKey logic, all query logic will be consolidated into the minimum number
+of range checks and equal checks by sending a Filter object along with the Scan
+with information about consolidated push down predicates
+
+.SparkSQL Code Example
+====
+This example shows how we can interact with HBase with SQL.
+
+[source, scala]
+----
+val sc = new SparkContext("local", "test")
+val config = new HBaseConfiguration()
+
+new HBaseContext(sc, TEST_UTIL.getConfiguration)
+val sqlContext = new SQLContext(sc)
+
+df = sqlContext.load("org.apache.hadoop.hbase.spark",
+  Map("hbase.columns.mapping" ->
+   "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD STRING c:b",
+   "hbase.table" -> "t1"))
+
+df.registerTempTable("hbaseTmp")
+
+val results = sqlContext.sql("SELECT KEY_FIELD, B_FIELD FROM hbaseTmp " +
+  "WHERE " +
+  "(KEY_FIELD = 'get1' and B_FIELD < '3') or " +
+  "(KEY_FIELD >= 'get3' and B_FIELD = '8')").take(5)
+----
+
+There are three major parts of this example that deserve explaining.
+
+The sqlContext.load function::
+  In the sqlContext.load function we see two
+  parameters. The first of these parameters is pointing Spark to the HBase
+  DefaultSource class that will act as the interface between SparkSQL and HBase.
+
+A map of key value pairs::
+  In this example we have two keys in our map, `hbase.columns.mapping` and
+  `hbase.table`. The `hbase.table` directs SparkSQL to use the given HBase table.
+  The `hbase.columns.mapping` key give us the logic to translate HBase columns to
+  SparkSQL columns.
++
+The `hbase.columns.mapping` is a string that follows the following format
++
+[source, scala]
+----
+(SparkSQL.ColumnName) (SparkSQL.ColumnType) (HBase.ColumnFamily):(HBase.Qualifier)
+----
++
+In the example below we see the definition of three fields. Because KEY_FIELD has
+no ColumnFamily, it is the RowKey.
++
+----
+KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD STRING c:b
+----
+
+The registerTempTable function::
+  This is a SparkSQL function that allows us now to be free of Scala when accessing
+  our HBase table directly with SQL with the table name of "hbaseTmp".
+
+The last major point to note in the example is the `sqlContext.sql` function, which
+allows the user to ask their questions in SQL which will be pushed down to the
+DefaultSource code in the HBase-Spark module. The result of this command will be
+a DataFrame with the Schema of KEY_FIELD and B_FIELD.
+====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c07ddc6d/src/main/asciidoc/_chapters/tracing.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc
index 9b3711e..0cddd8a 100644
--- a/src/main/asciidoc/_chapters/tracing.adoc
+++ b/src/main/asciidoc/_chapters/tracing.adoc
@@ -31,12 +31,12 @@
 :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://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). 
+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 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: 
+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,12 +45,12 @@ 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]
 ----
@@ -65,7 +65,7 @@ The `LocalFileSpanReceiver` looks in _hbase-site.xml_      for a `hbase.local-fi
 </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 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.
 
@@ -77,11 +77,11 @@ _htrace-zipkin_ is published to the link:http://search.maven.org/#search%7Cgav%7
 <property>
   <name>hbase.trace.spanreceiver.classes</name>
   <value>org.apache.htrace.impl.ZipkinSpanReceiver</value>
-</property> 
+</property>
 <property>
   <name>hbase.htrace.zipkin.collector-hostname</name>
   <value>localhost</value>
-</property> 
+</property>
 <property>
   <name>hbase.htrace.zipkin.collector-port</name>
   <value>9410</value>
@@ -93,7 +93,7 @@ If you do not want to use the included span receivers, you are encouraged to wri
 [[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]
 ----
@@ -107,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]
 ----
@@ -118,7 +118,7 @@ Get get = new Get(Bytes.toBytes("r1"));
 Result res = table.get(get);
 ----
 
-into: 
+into:
 
 [source,java]
 ----
@@ -133,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]
 ----
@@ -142,12 +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]
 ----
@@ -158,7 +158,7 @@ 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 success. `trace 'status'` just returns if or not tracing is turned on. 
+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/c07ddc6d/src/main/asciidoc/_chapters/troubleshooting.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/troubleshooting.adoc b/src/main/asciidoc/_chapters/troubleshooting.adoc
index 8ae61b4..e372760 100644
--- a/src/main/asciidoc/_chapters/troubleshooting.adoc
+++ b/src/main/asciidoc/_chapters/troubleshooting.adoc
@@ -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]
@@ -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]
 ----
@@ -849,7 +849,7 @@ 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 tne snapshot. This includes deleted cells or expired versions.
+  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
@@ -1070,7 +1070,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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c07ddc6d/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..6f13864 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,13 @@ 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 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 +133,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 +150,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 +182,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 +197,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 +225,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 +242,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 +259,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.
@@ -283,7 +283,7 @@ Check the versions to be sure they are appropriate.
     <type>test-jar</type>
     <scope>test</scope>
 </dependency>
-        
+
 <dependency>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-hdfs</artifactId>
@@ -309,7 +309,7 @@ public class MyHBaseIntegrationTest {
     private static HBaseTestingUtility utility;
     byte[] CF = "CF".getBytes();
     byte[] QUALIFIER = "CQ-1".getBytes();
-    
+
     @Before
     public void setup() throws Exception {
     	utility = new HBaseTestingUtility();
@@ -343,7 +343,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/c07ddc6d/src/main/asciidoc/_chapters/upgrading.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index 13c3c0e..6327c5a 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -132,7 +132,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 +158,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]
 ----
@@ -200,7 +200,7 @@ 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.
@@ -392,7 +392,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/c07ddc6d/src/main/asciidoc/_chapters/zookeeper.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/zookeeper.adoc b/src/main/asciidoc/_chapters/zookeeper.adoc
index 3266964..2319360 100644
--- a/src/main/asciidoc/_chapters/zookeeper.adoc
+++ b/src/main/asciidoc/_chapters/zookeeper.adoc
@@ -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,12 +97,12 @@ 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_. 
+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_.
 ====
 
 .ZooKeeper Maintenance
@@ -140,7 +140,7 @@ 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
@@ -148,24 +148,24 @@ Additionally, see the link:http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7[ZooKee
 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:
 
@@ -210,7 +210,7 @@ where the _$PATH_TO_HBASE_KEYTAB_ and _$PATH_TO_ZOOKEEPER_KEYTAB_ files are what
 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:
 
@@ -257,7 +257,7 @@ Modify your _hbase-site.xml_ on each node that will run zookeeper, master or reg
 
 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:
 
 ----
 
@@ -344,7 +344,7 @@ 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:
 
@@ -354,7 +354,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:
 
 ----
 
@@ -415,7 +415,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):
 
 ----
 
@@ -439,7 +439,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/c07ddc6d/src/main/asciidoc/book.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index b2bd151..392ba2e 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -64,6 +64,7 @@ include::_chapters/architecture.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[]


Mime
View raw message