You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2018/08/14 21:52:37 UTC

[2/2] hbase git commit: HBASE-21054 Copy down docs, amend to suite branch-2.0, and then commit

HBASE-21054 Copy down docs, amend to suite branch-2.0, and then commit


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

Branch: refs/heads/branch-2.0
Commit: d49fca134415ed02ceb06c007ffc4734a27958a6
Parents: 84b54f4
Author: Michael Stack <st...@apache.org>
Authored: Tue Aug 14 14:51:08 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 14 14:52:13 2018 -0700

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/amv2.adoc           | 173 +++++
 .../asciidoc/_chapters/appendix_acl_matrix.adoc |   1 +
 .../appendix_hbase_incompatibilities.adoc       | 714 +++++++++++++++++++
 src/main/asciidoc/_chapters/architecture.adoc   |  87 +++
 src/main/asciidoc/_chapters/community.adoc      |   3 +
 src/main/asciidoc/_chapters/configuration.adoc  |  16 +-
 src/main/asciidoc/_chapters/external_apis.adoc  | 101 ++-
 src/main/asciidoc/_chapters/ops_mgt.adoc        | 166 ++++-
 src/main/asciidoc/_chapters/preface.adoc        |   2 +-
 src/main/asciidoc/_chapters/pv2.adoc            | 163 +++++
 src/main/asciidoc/_chapters/security.adoc       |  22 +-
 .../asciidoc/_chapters/sync_replication.adoc    | 125 ++++
 .../asciidoc/_chapters/troubleshooting.adoc     |  55 +-
 src/main/asciidoc/_chapters/upgrading.adoc      |  47 +-
 src/main/asciidoc/book.adoc                     |   3 +
 15 files changed, 1578 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/src/main/asciidoc/_chapters/amv2.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/amv2.adoc b/src/main/asciidoc/_chapters/amv2.adoc
new file mode 100644
index 0000000..49841ce
--- /dev/null
+++ b/src/main/asciidoc/_chapters/amv2.adoc
@@ -0,0 +1,173 @@
+////
+/**
+ *
+ * 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.
+ */
+////
+[[amv2]]
+= AMv2 Description for Devs
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+The AssignmentManager (AM) in HBase Master manages assignment of Regions over a cluster of RegionServers.
+
+The AMv2 project is a redo of Assignment in an attempt at addressing the root cause of many of our operational issues in production, namely slow assignment and problematic accounting such that Regions are misplaced stuck offline in the notorious _Regions-In-Transition (RIT)_ limbo state.
+
+Below are notes for devs on key aspects of AMv2 in no particular order.
+
+== Background
+
+Assignment in HBase 1.x has been problematic in operation. It is not hard to see why. Region state is kept at the other end of an RPC in ZooKeeper (Terminal states -- i.e. OPEN or CLOSED -- are published to the _hbase:meta_ table). In HBase-1.x.x, state has multiple writers with Master and RegionServers all able to make state edits concurrently (in _hbase:meta_ table and out on ZooKeeper). If clocks are awry or watchers missed, state changes can be skipped or overwritten. Locking of HBase Entities -- tables, regions -- is not comprehensive so a table operation -- disable/enable -- could clash with a region-level operation; a split or merge. Region state is distributed and hard to reason about and test. Assignment is slow in operation because each assign involves moving remote znodes through transitions. Cluster size tends to top out at a couple of hundred thousand regions; beyond this, cluster start/stop takes hours and is prone to corruption.
+
+AMv2 (AssignmentManager Version 2) is a refactor (https://issues.apache.org/jira/browse/HBASE-14350[HBASE-14350]) of the hbase-1.x AssignmentManager putting it up on a https://issues.apache.org/jira/browse/HBASE-12439[ProcedureV2 (HBASE-12439)] basis. ProcedureV2 (Pv2)__,__ is an awkwardly named system that allows describing and running multi-step state machines. It is performant and persists all state to a Store which is recoverable post crash. See the companion chapter on <<pv2>>, to learn more about the ProcedureV2 system.
+
+In AMv2, all assignment, crash handling, splits and merges are recast as Procedures(v2).  ZooKeeper is purged from the mix. As before, the final assignment state gets published to _hbase:meta_ for non-Master participants to read (all-clients) with intermediate state kept in the local Pv2 WAL-based ‘store’ but only the active Master, a single-writer, evolves state. The Master’s in-memory cluster image is the authority and if disagreement, RegionServers are forced to comply. Pv2 adds shared/exclusive locking of all core HBase Entities -- namespace, tables, and regions -- to ensure one actor at a time access and to prevent operations contending over resources (move/split, disable/assign, etc.).
+
+This redo of AM atop of a purposed, performant state machine with all operations taking on the common Procedure form with a single state writer only moves our AM to a new level of resilience and scale.
+
+== New System
+
+Each Region Assign or Unassign of a Region is now a Procedure. A Move (Region) Procedure is a compound of Procedures; it is the running of an Unassign Procedure followed by an Assign Procedure. The Move Procedure spawns the Assign and Unassign in series and then waits on their completions.
+
+And so on. ServerCrashProcedure spawns the WAL splitting tasks and then the reassign of all regions that were hosted on the crashed server as subprocedures.
+
+AMv2 Procedures are run by the Master in a ProcedureExecutor instance. All Procedures make use of utility provided by the Pv2 framework.
+
+For example, Procedures persist each state transition to the frameworks’ Procedure Store. The default implementation is done as a WAL kept on HDFS. On crash, we reopen the Store and rerun all WALs of Procedure transitions to put the Assignment State Machine back into the attitude it had just before crash. We then continue Procedure execution.
+
+In the new system, the Master is the Authority on all things Assign. Previous we were ambiguous; e.g. the RegionServer was in charge of Split operations. Master keeps an in-memory image of Region states and servers. If disagreement, the Master always prevails; at an extreme it will kill the RegionServer that is in disagreement.
+
+A new RegionStateStore class takes care of publishing the terminal Region state, whether OPEN or CLOSED, out to the _hbase:meta _table__.__
+
+RegionServers now report their run version on Connection. This version is available inside the AM for use running migrating rolling restarts.
+
+== Procedures Detail
+
+=== Assign/Unassign
+
+Assign and Unassign subclass a common RegionTransitionProcedure. There can only be one RegionTransitionProcedure per region running at a time since the RTP instance takes a lock on the region. The RTP base Procedure has three steps; a store the procedure step (REGION_TRANSITION_QUEUE); a dispatch of the procedure open or close followed by a suspend waiting on the remote regionserver to report successful open or fail (REGION_TRANSITION_DISPATCH) or notification that the server fielding the request crashed; and finally registration of the successful open/close in hbase:meta (REGION_TRANSITION_FINISH).
+
+Here is how the assign of a region 56f985a727afe80a184dac75fbf6860c looks in the logs. The assign was provoked by a Server Crash (Process ID 1176 or pid=1176 which when it is the parent of a procedure, it is identified as ppid=1176). The assign is pid=1179, the second region of the two being assigned by this Server Crash.
+
+[source]
+----
+2017-05-23 12:04:24,175 INFO  [ProcExecWrkr-30] procedure2.ProcedureExecutor: Initialized subprocedures=[{pid=1178, ppid=1176, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=bfd57f0b72fd3ca77e9d3c5e3ae48d76, target=ve0540.halxg.example.org,16020,1495525111232}, {pid=1179, ppid=1176, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=56f985a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,16020,1495525111232}]
+----
+
+Next we start the assign by queuing (‘registering’) the Procedure with the framework.
+
+[source]
+----
+2017-05-23 12:04:24,241 INFO  [ProcExecWrkr-30] assignment.AssignProcedure: Start pid=1179, ppid=1176, state=RUNNABLE:REGION_TRANSITION_QUEUE; AssignProcedure table=IntegrationTestBigLinkedList, region=56f985a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,16020,1495525111232; rit=OFFLINE, location=ve0540.halxg.example.org,16020,1495525111232; forceNewPlan=false, retain=false
+----
+
+Track the running of Procedures in logs by tracing their process id -- here pid=1179.
+
+Next we move to the dispatch phase where we update hbase:meta table setting the region state as OPENING on server ve540. We then dispatch an rpc to ve540 asking it to open the region. Thereafter we suspend the Assign until we get a message back from ve540 on whether it has opened the region successfully (or not).
+
+[source]
+----
+2017-05-23 12:04:24,494 INFO  [ProcExecWrkr-38] assignment.RegionStateStore: pid=1179 updating hbase:meta row=IntegrationTestBigLinkedList,H\xE3@\x8D\x964\x9D\xDF\x8F@9\x0F\xC8\xCC\xC2,1495566261066.56f985a727afe80a184dac75fbf6860c., regionState=OPENING, regionLocation=ve0540.halxg.example.org,16020,1495525111232
+2017-05-23 12:04:24,498 INFO  [ProcExecWrkr-38] assignment.RegionTransitionProcedure: Dispatch pid=1179, ppid=1176, state=RUNNABLE:REGION_TRANSITION_DISPATCH; AssignProcedure table=IntegrationTestBigLinkedList, region=56f985a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,16020,1495525111232; rit=OPENING, location=ve0540.halxg.example.org,16020,1495525111232
+----
+
+Below we log the incoming report that the region opened successfully on ve540. The Procedure is woken up (you can tell it the procedure is running by the name of the thread, its a ProcedureExecutor thread, ProcExecWrkr-9).  The woken up Procedure updates state in hbase:meta to denote the region as open on ve0540. It then reports finished and exits.
+
+[source]
+----
+2017-05-23 12:04:26,643 DEBUG [RpcServer.default.FPBQ.Fifo.handler=46,queue=1,port=16000] assignment.RegionTransitionProcedure: Received report OPENED seqId=11984985, pid=1179, ppid=1176, state=RUNNABLE:REGION_TRANSITION_DISPATCH; AssignProcedure table=IntegrationTestBigLinkedList, region=56f985a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,16020,1495525111232; rit=OPENING, location=ve0540.halxg.example.org,16020,1495525111232                                                                                                                                                                       2017-05-23 12:04:26,643 INFO  [ProcExecWrkr-9] assignment.RegionStateStore: pid=1179 updating hbase:meta row=IntegrationTestBigLinkedList,H\xE3@\x8D\x964\x9D\xDF\x8F@9\x0F\xC8\xCC\xC2,1495566261066.56f985a727afe80a184dac75fbf6860c., regionState=OPEN, openSeqNum=11984985, regionLocation=ve0540.halxg.example.org,16020,1495525111232
+2017-05-23 12:04:26,836 INFO  [ProcExecWrkr-9] procedure2.ProcedureExecutor: Finish suprocedure pid=1179, ppid=1176, state=SUCCESS; AssignProcedure table=IntegrationTestBigLinkedList, region=56f985a727afe80a184dac75fbf6860c, target=ve0540.halxg.example.org,16020,1495525111232
+----
+Unassign looks similar given it is based on the base RegionTransitionProcedure. It has the same state transitions and does basically the same steps but with different state name (CLOSING, CLOSED).
+
+Most other procedures are subclasses of a Pv2 StateMachine implementation. We have both Table and Region focused StateMachines types.
+
+== UI
+
+Along the top-bar on the Master, you can now find a ‘Procedures&Locks’ tab which takes you to a page that is ugly but useful. It dumps currently running procedures and framework locks. Look at this when you can’t figure what stuff is stuck; it will at least identify problematic procedures (take the pid and grep the logs…). Look for ROLLEDBACK or pids that have been RUNNING for a long time.
+
+== Logging
+
+Procedures log their process ids as pid= and their parent ids (ppid=) everywhere. Work has been done so you can grep the pid and see history of a procedure operation.
+
+== Implementation Notes
+
+In this section we note some idiosyncrasies of operation as an attempt at saving you some head-scratching.
+
+=== Region Transition RPC and RS Heartbeat can arrive at ~same time on Master
+
+Reporting Region Transition on a RegionServer is now a RPC distinct from RS heartbeating (‘RegionServerServices’ Service). An heartbeat and a status update can arrive at the Master at about the same time. The Master will update its internal state for a Region but this same state is checked when heartbeat processing. We may find the unexpected; i.e. a Region just reported as CLOSED so heartbeat is surprised to find region OPEN on the back of the RS report. In the new system, all slaves must cow to the Masters’ understanding of cluster state; the Master will kill/close any misaligned entities.
+
+To address the above, we added a lastUpdate for in-memory Master state. Let a region state have some vintage before we act on it (one second currently).
+
+=== Master as RegionServer or as RegionServer that just does system tables
+
+AMv2 enforces current master branch default of HMaster carrying system tables only; i.e. the Master in an HBase cluster acts also as a RegionServer only it is the exclusive host for tables such as _hbase:meta_, _hbase:namespace_, etc., the core system tables. This is causing a couple of test failures as AMv1, though it is not supposed to, allows moving hbase:meta off Master while AMv2 does not.
+
+== New Configs
+
+These configs all need doc on when you’d change them.
+
+=== hbase.procedure.remote.dispatcher.threadpool.size
+
+Defaults 128
+
+=== hbase.procedure.remote.dispatcher.delay.msec
+
+Default 150ms
+
+=== hbase.procedure.remote.dispatcher.max.queue.size
+
+Default 32
+
+=== hbase.regionserver.rpc.startup.waittime
+
+Default 60 seconds.
+
+== Tools
+
+HBASE-15592 Print Procedure WAL Content
+
+Patch in https://issues.apache.org/jira/browse/HBASE-18152[HBASE-18152] [AMv2] Corrupt Procedure WAL file; procedure data stored out of order https://issues.apache.org/jira/secure/attachment/12871066/reading_bad_wal.patch[https://issues.apache.org/jira/secure/attachment/12871066/reading_bad_wal.patch]
+
+=== MasterProcedureSchedulerPerformanceEvaluation
+
+Tool to test performance of locks and queues in procedure scheduler independently from other framework components. Run this after any substantial changes in proc system. Prints nice output:
+
+----
+******************************************
+Time - addBack     : 5.0600sec
+Ops/sec - addBack  : 1.9M
+Time - poll        : 19.4590sec
+Ops/sec - poll     : 501.9K
+Num Operations     : 10000000
+
+Completed          : 10000006
+Yield              : 22025876
+
+Num Tables         : 5
+Regions per table  : 10
+Operations type    : both
+Threads            : 10
+******************************************
+Raw format for scripts
+
+RESULT [num_ops=10000000, ops_type=both, num_table=5, regions_per_table=10, threads=10, num_yield=22025876, time_addback_ms=5060, time_poll_ms=19459]
+----

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/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 d5ea076..cb17346 100644
--- a/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
+++ b/src/main/asciidoc/_chapters/appendix_acl_matrix.adoc
@@ -160,6 +160,7 @@ In case the table goes out of date, the unit tests which check for accuracy of p
 |                  | getUserPermissions(global level) | global(A)
 |                  | getUserPermissions(namespace level) | global(A)\|NS(A)
 |                  | getUserPermissions(table level) | global(A)\|NS(A)\|TableOwner\|table(A)\|CF(A)\|CQ(A)
+|                  | hasPermission(table level) | global(A)\|SelfUserCheck
 | RegionServer | stopRegionServer | superuser\|global(A)
 |              | mergeRegions | superuser\|global(A)
 |              | rollWALWriterRequest | superuser\|global(A)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/src/main/asciidoc/_chapters/appendix_hbase_incompatibilities.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_hbase_incompatibilities.adoc b/src/main/asciidoc/_chapters/appendix_hbase_incompatibilities.adoc
new file mode 100644
index 0000000..d450f04
--- /dev/null
+++ b/src/main/asciidoc/_chapters/appendix_hbase_incompatibilities.adoc
@@ -0,0 +1,714 @@
+////
+/**
+ *
+ * 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.
+ */
+////
+
+[appendix]
+== Known Incompatibilities Among HBase Versions
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+:toc: left
+:source-language: java
+
+== HBase 2.0 Incompatible Changes
+
+This appendix describes incompatible changes from earlier versions of HBase against HBase 2.0.
+This list is not meant to be wholly encompassing of all possible incompatibilities.
+Instead, this content is intended to give insight into some obvious incompatibilities which most
+users will face coming from HBase 1.x releases.
+
+=== List of Major Changes for HBase 2.0
+* HBASE-1912- HBCK is a HBase database checking tool for capturing the inconsistency. As an HBase administrator, you should not use HBase version 1.0  hbck tool to check the HBase 2.0 database. Doing so will break the database and throw an exception error.
+* HBASE-16189 and HBASE-18945- You cannot open the HBase 2.0 hfiles through HBase 1.0 version.  If you are an admin or an HBase user who is using HBase version 1.x, you must first do a rolling upgrade to the latest version of HBase 1.x and then upgrade to HBase 2.0.
+* HBASE-18240 - Changed the ReplicationEndpoint Interface. It also introduces a new hbase-third party 1.0 that packages all the third party utilities, which are expected to run in the hbase cluster.
+
+=== Coprocessor API changes
+
+* HBASE-16769 - Deprecated PB references from MasterObserver and RegionServerObserver.
+* HBASE-17312 - [JDK8] Use default method for Observer Coprocessors. The interface classes of BaseMasterAndRegionObserver, BaseMasterObserver, BaseRegionObserver, BaseRegionServerObserver and BaseWALObserver uses JDK8's 'default' keyword to provide empty and no-op implementations.
+* Interface HTableInterface
+  HBase 2.0 introduces following changes to the methods listed below:
+
+==== [−] interface CoprocessorEnvironment changes (2)
+
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method getTable ( TableName ) has been removed. | A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getTable ( TableName, ExecutorService ) has been removed. | A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* Public Audience
+
+The following tables describes the coprocessor changes.
+
+===== [−] class CoprocessorRpcChannel  (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| This class has become interface.| A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception depending on the usage of this class.
+|===
+
+===== Class CoprocessorHost<E>
+Classes that were Audience Private but were removed.
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Type of field coprocessors has been changed from java.util.SortedSet<E> to org.apache.hadoop.hbase.util.SortedList<E>.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+
+==== MasterObserver
+HBase 2.0 introduces following changes to the MasterObserver interface.
+
+===== [−] interface MasterObserver  (14)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method voidpostCloneSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostCreateTable ( ObserverContext<MasterCoprocessorEnvironment>, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpostDeleteSnapshot (ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpostGetTableDescriptors ( ObserverContext<MasterCoprocessorEnvironment>, List<HTableDescriptor> ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpostModifyTable ( ObserverContext<MasterCoprocessorEnvironment>, TableName, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpostRestoreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpostSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreCloneSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreCreateTable ( ObserverContext<MasterCoprocessorEnvironment>, HTableDescriptor, HRegionInfo[ ] ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreDeleteSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreGetTableDescriptors ( ObserverContext<MasterCoprocessorEnvironment>, List<TableName>, List<HTableDescriptor> ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreModifyTable ( ObserverContext<MasterCoprocessorEnvironment>, TableName, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreRestoreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+| Abstract method voidpreSnapshot ( ObserverContext<MasterCoprocessorEnvironment>, HBaseProtos.SnapshotDescription, HTableDescriptor ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodErrorexception.
+|===
+
+==== RegionObserver
+HBase 2.0 introduces following changes to the RegionObserver interface.
+
+===== [−] interface RegionObserver  (13)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method voidpostCloseRegionOperation ( ObserverContext<RegionCoprocessorEnvironment>, HRegion.Operation ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostCompactSelection ( ObserverContext<RegionCoprocessorEnvironment>, Store, ImmutableList<StoreFile> ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostCompactSelection ( ObserverContext<RegionCoprocessorEnvironment>, Store, ImmutableList<StoreFile>, CompactionRequest ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostGetClosestRowBefore ( ObserverContext<RegionCoprocessorEnvironment>, byte[ ], byte[ ], Result ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method DeleteTrackerpostInstantiateDeleteTracker ( ObserverContext<RegionCoprocessorEnvironment>, DeleteTracker ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostSplit ( ObserverContext<RegionCoprocessorEnvironment>, HRegion, HRegion ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostStartRegionOperation ( ObserverContext<RegionCoprocessorEnvironment>, HRegion.Operation ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method StoreFile.ReaderpostStoreFileReaderOpen ( ObserverContext<RegionCoprocessorEnvironment>, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpostWALRestore ( ObserverContext<RegionCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method InternalScannerpreFlushScannerOpen ( ObserverContext<RegionCoprocessorEnvironment>, Store, KeyValueScanner, InternalScanner ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpreGetClosestRowBefore ( ObserverContext<RegionCoprocessorEnvironment>, byte[ ], byte[ ], Result ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method StoreFile.ReaderpreStoreFileReaderOpen ( ObserverContext<RegionCoprocessorEnvironment>, FileSystem, Path, FSDataInputStreamWrapper, long, CacheConfig, Reference, StoreFile.Reader ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method voidpreWALRestore ( ObserverContext<RegionCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== WALObserver
+HBase 2.0 introduces following changes to the WALObserver interface.
+
+====== [−] interface WALObserver
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method voidpostWALWrite ( ObserverContext<WALCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method booleanpreWALWrite ( ObserverContext<WALCoprocessorEnvironment>, HRegionInfo, HLogKey, WALEdit ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== Miscellaneous
+HBase 2.0 introduces changes to the following classes:
+
+hbase-server-1.0.0.jar, OnlineRegions.class package org.apache.hadoop.hbase.regionserver
+[cols="1,1", frame="all"]
+===== [−] OnlineRegions.getFromOnlineRegions ( String p1 ) [abstract]  :  HRegion
+org/apache/hadoop/hbase/regionserver/OnlineRegions.getFromOnlineRegions:(Ljava/lang/String;)Lorg/apache/hadoop/hbase/regionserver/HRegion;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from Region to Region.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+hbase-server-1.0.0.jar, RegionCoprocessorEnvironment.class package org.apache.hadoop.hbase.coprocessor
+
+===== [−] RegionCoprocessorEnvironment.getRegion ( ) [abstract]  : HRegion
+org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.getRegion:()Lorg/apache/hadoop/hbase/regionserver/HRegion;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.regionserver.HRegion to org.apache.hadoop.hbase.regionserver.Region.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+hbase-server-1.0.0.jar, RegionCoprocessorHost.class package org.apache.hadoop.hbase.regionserver
+
+===== [−] RegionCoprocessorHost.postAppend ( Append append, Result result )  : void
+org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.postAppend:(Lorg/apache/hadoop/hbase/client/Append;Lorg/apache/hadoop/hbase/client/Result;)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to org.apache.hadoop.hbase.client.Result.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] RegionCoprocessorHost.preStoreFileReaderOpen ( FileSystem fs, Path p,   FSDataInputStreamWrapper in, long size,CacheConfig cacheConf, Reference r )  :  StoreFile.Reader
+org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.preStoreFileReaderOpen:(Lorg/apache/hadoop/fs/FileSystem;Lorg/apache/hadoop/fs/Path;Lorg/apache/hadoop/hbase/io/FSDataInputStreamWrapper;JLorg/apache/hadoop/hbase/io/hfile/CacheConfig;Lorg/apache/hadoop/hbase/io/Reference;)Lorg/apache/hadoop/hbase/regionserver/StoreFile$Reader;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from StoreFile.Reader to StoreFileReader.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== IPC
+==== Scheduler changes:
+1. Following methods became abstract:
+
+package org.apache.hadoop.hbase.ipc
+
+===== [−]class RpcScheduler (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method void dispatch ( CallRunner ) has been removed from this class.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+hbase-server-1.0.0.jar, RpcScheduler.class package org.apache.hadoop.hbase.ipc
+
+===== [−] RpcScheduler.dispatch ( CallRunner p1 ) [abstract]  :  void  1
+org/apache/hadoop/hbase/ipc/RpcScheduler.dispatch:(Lorg/apache/hadoop/hbase/ipc/CallRunner;)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to boolean.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+2. Following abstract methods have been removed:
+
+===== [−]interface PriorityFunction  (2)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method longgetDeadline ( RPCProtos.RequestHeader, Message ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method int getPriority ( RPCProtos.RequestHeader, Message ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== Server API changes:
+
+===== [−] class RpcServer  (12)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Type of field CurCall has been changed from java.lang.ThreadLocal<RpcServer.Call> to java.lang.ThreadLocal<RpcCall>.| A client program may be interrupted by NoSuchFieldError exception.
+| This class became abstract.| A client program may be interrupted by InstantiationError exception.
+| Abstract method int getNumOpenConnections ( ) has been added to this class.| This class became abstract and a client program may be interrupted by InstantiationError exception.
+| Field callQueueSize of type org.apache.hadoop.hbase.util.Counter has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field connectionList of type java.util.List<RpcServer.Connection> has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field maxIdleTime of type int has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field numConnections of type int has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field port of type int has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field purgeTimeout of type long has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field responder of type RpcServer.Responder has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field socketSendBufferSize of type int has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field thresholdIdleConnections of type int has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+Following abstract method has been removed:
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method Pair<Message,CellScanner>call ( BlockingService, Descriptors.MethodDescriptor, Message, CellScanner, long, MonitoredRPCHandler ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== Replication and WAL changes:
+HBASE-18733: WALKey has been purged completely in HBase 2.0.
+Following are the changes to the WALKey:
+
+===== [−] classWALKey (8)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Access level of field clusterIds has been changed from protected to private.| A client program may be interrupted by IllegalAccessError exception.
+| Access level of field compressionContext has been changed from protected to private.| A client program may be interrupted by IllegalAccessError exception.
+| Access level of field encodedRegionName has been changed from protected to private.| A client program may be interrupted by IllegalAccessError exception.
+| Access level of field tablename has been changed from protectedto private.| A client program may be interrupted by IllegalAccessError exception.
+| Access level of field writeTime has been changed from protectedto private.| A client program may be interrupted by IllegalAccessError exception.
+|===
+
+Following fields have been removed:
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Field LOG of type org.apache.commons.logging.Log has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field VERSION of type WALKey.Version has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field logSeqNum of type long has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+Following are the changes to the WALEdit.class:
+hbase-server-1.0.0.jar, WALEdit.class package org.apache.hadoop.hbase.regionserver.wal
+
+===== WALEdit.getCompaction ( Cell kv ) [static]  :  WALProtos.CompactionDescriptor  (1)
+org/apache/hadoop/hbase/regionserver/wal/WALEdit.getCompaction:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$CompactionDescriptor;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== WALEdit.getFlushDescriptor ( Cell cell ) [static]  :  WALProtos.FlushDescriptor  (1)
+org/apache/hadoop/hbase/regionserver/wal/WALEdit.getFlushDescriptor:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$FlushDescriptor;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== WALEdit.getRegionEventDescriptor ( Cell cell ) [static]  :  WALProtos.RegionEventDescriptor  (1)
+org/apache/hadoop/hbase/regionserver/wal/WALEdit.getRegionEventDescriptor:(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$RegionEventDescriptor;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+Following is the change to the WALKey.class:
+package org.apache.hadoop.hbase.wal
+
+===== WALKey.getBuilder ( WALCellCodec.ByteStringCompressor compressor )  :  WALProtos.WALKey.Builder  1
+org/apache/hadoop/hbase/wal/WALKey.getBuilder:(Lorg/apache/hadoop/hbase/regionserver/wal/WALCellCodec$ByteStringCompressor;)Lorg/apache/hadoop/hbase/protobuf/generated/WALProtos$WALKey$Builder;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey.Builder to org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey.Builder.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== Deprecated APIs or coprocessor:
+
+HBASE-16769 - PB references from MasterObserver and RegionServerObserver has been removed.
+
+==== Admin Interface API changes:
+You cannot administer an HBase 2.0 cluster with an HBase 1.0 client that includes RelicationAdmin, ACC, Thrift and REST usage of Admin ops. Methods returning protobufs have been changed to return POJOs instead. pb is not used in the APIs anymore. Returns have changed from void to Future for async methods.
+HBASE-18106 - Admin.listProcedures and Admin.listLocks were renamed to getProcedures and getLocks.
+MapReduce makes use of Admin doing following admin.getClusterStatus() to calcluate Splits.
+
+* Thrift usage of Admin API:
+compact(ByteBuffer)
+createTable(ByteBuffer, List<ColumnDescriptor>)
+deleteTable(ByteBuffer)
+disableTable(ByteBuffer)
+enableTable(ByteBuffer)
+getTableNames()
+majorCompact(ByteBuffer)
+
+* REST usage of Admin API:
+hbase-rest
+org.apache.hadoop.hbase.rest
+RootResource
+getTableList()
+    TableName[] tableNames = servlet.getAdmin().listTableNames();
+SchemaResource
+delete(UriInfo)
+      Admin admin = servlet.getAdmin();
+update(TableSchemaModel, boolean, UriInfo)
+      Admin admin = servlet.getAdmin();
+StorageClusterStatusResource
+get(UriInfo)
+      ClusterStatus status = servlet.getAdmin().getClusterStatus();
+StorageClusterVersionResource
+get(UriInfo)
+      model.setVersion(servlet.getAdmin().getClusterStatus().getHBaseVersion());
+TableResource
+exists()
+    return servlet.getAdmin().tableExists(TableName.valueOf(table));
+
+Following are the changes to the Admin interface:
+
+===== [−] interface Admin (9)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method createTableAsync ( HTableDescriptor, byte[ ][ ] ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method disableTableAsync ( TableName ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method enableTableAsync ( TableName ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getCompactionState ( TableName ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getCompactionStateForRegion ( byte[ ] ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method isSnapshotFinished ( HBaseProtos.SnapshotDescription ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method snapshot ( String, TableName, HBaseProtos.SnapshotDescription.Type ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method snapshot ( HBaseProtos.SnapshotDescription ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method takeSnapshotAsync ( HBaseProtos.SnapshotDescription ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+Following are the changes to the Admin.class:
+hbase-client-1.0.0.jar, Admin.class package org.apache.hadoop.hbase.client
+
+===== [−] Admin.createTableAsync ( HTableDescriptor p1, byte[ ][ ] p2 ) [abstract]  :  void  1
+org/apache/hadoop/hbase/client/Admin.createTableAsync:(Lorg/apache/hadoop/hbase/HTableDescriptor;[[B)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] Admin.disableTableAsync ( TableName p1 ) [abstract]  :  void  1
+org/apache/hadoop/hbase/client/Admin.disableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== Admin.enableTableAsync ( TableName p1 ) [abstract]  :  void  1
+org/apache/hadoop/hbase/client/Admin.enableTableAsync:(Lorg/apache/hadoop/hbase/TableName;)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to java.util.concurrent.Future<java.lang.Void>.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] Admin.getCompactionState ( TableName p1 ) [abstract]  :  AdminProtos.GetRegionInfoResponse.CompactionState  1
+org/apache/hadoop/hbase/client/Admin.getCompactionState:(Lorg/apache/hadoop/hbase/TableName;)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] Admin.getCompactionStateForRegion ( byte[ ] p1 ) [abstract]  :  AdminProtos.GetRegionInfoResponse.CompactionState  1
+org/apache/hadoop/hbase/client/Admin.getCompactionStateForRegion:([B)Lorg/apache/hadoop/hbase/protobuf/generated/AdminProtos$GetRegionInfoResponse$CompactionState;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState to CompactionState.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== HTableDescriptor and HColumnDescriptor changes
+HTableDescriptor and HColumnDescriptor has become interfaces and you can create it through Builders. HCD has become CFD. It no longer implements writable interface.
+package org.apache.hadoop.hbase
+
+===== [−] class HColumnDescriptor  (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Removed super-interface org.apache.hadoop.io.WritableComparable<HColumnDescriptor>.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+HColumnDescriptor in 1.0.0
+{code}
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HColumnDescriptor implements WritableComparable<HColumnDescriptor> {
+{code}
+
+HColumnDescriptor in 2.0
+{code}
+@InterfaceAudience.Public
+@Deprecated // remove it in 3.0
+public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HColumnDescriptor> {
+{code}
+
+For META_TABLEDESC, the maker method had been deprecated already in HTD in 1.0.0.  OWNER_KEY is still in HTD.
+
+===== class HTableDescriptor  (3)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Removed super-interface org.apache.hadoop.io.WritableComparable<HTableDescriptor>.| A client program may be interrupted by NoSuchMethodError exception.
+| Field META_TABLEDESC of type HTableDescriptor has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+hbase-client-1.0.0.jar, HTableDescriptor.class package org.apache.hadoop.hbase
+
+===== [−] HTableDescriptor.getColumnFamilies ( )  :  HColumnDescriptor[ ]  (1)
+org/apache/hadoop/hbase/HTableDescriptor.getColumnFamilies:()[Lorg/apache/hadoop/hbase/HColumnDescriptor;
+
+===== [−] class HColumnDescriptor  (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from HColumnDescriptor[]to client.ColumnFamilyDescriptor[].| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] HTableDescriptor.getCoprocessors ( )  :  List<String>  (1)
+org/apache/hadoop/hbase/HTableDescriptor.getCoprocessors:()Ljava/util/List;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from java.util.List<java.lang.String> to java.util.Collection.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* HBASE-12990 MetaScanner is removed and it is replaced by MetaTableAccessor.
+
+===== HTableWrapper changes:
+hbase-server-1.0.0.jar, HTableWrapper.class package org.apache.hadoop.hbase.client
+
+===== [−] HTableWrapper.createWrapper ( List<HTableInterface> openTables, TableName tableName, CoprocessorHost.Environment env, ExecutorService pool ) [static]  :  HTableInterface  1
+org/apache/hadoop/hbase/client/HTableWrapper.createWrapper:(Ljava/util/List;Lorg/apache/hadoop/hbase/TableName;Lorg/apache/hadoop/hbase/coprocessor/CoprocessorHost$Environment;Ljava/util/concurrent/ExecutorService;)Lorg/apache/hadoop/hbase/client/HTableInterface;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from HTableInterface to Table.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* HBASE-12586: Delete all public HTable constructors and delete ConnectionManager#{delete,get}Connection.
+* HBASE-9117: Remove HTablePool and all HConnection pooling related APIs.
+* HBASE-13214: Remove deprecated and unused methods from HTable class
+Following are the changes to the Table interface:
+
+===== [−] interface Table  (4)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method batch ( List<?> ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method batchCallback ( List<?>, Batch.Callback<R> )has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getWriteBufferSize ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method setWriteBufferSize ( long ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== Deprecated buffer methods in Table (in 1.0.1) and removed in 2.0.0
+
+* HBASE-13298- Clarify if Table.{set|get}WriteBufferSize() is deprecated or not.
+
+* LockTimeoutException and OperationConflictException classes have been removed.
+
+==== class OperationConflictException  (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| This class has been removed.| A client program may be interrupted by NoClassDefFoundErrorexception.
+|===
+
+==== class class LockTimeoutException  (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| This class has been removed.| A client program may be interrupted by NoClassDefFoundErrorexception.
+|===
+
+==== Filter API changes:
+Following methods have been removed:
+package org.apache.hadoop.hbase.filter
+
+===== [−] class Filter  (2)
+|===
+| Change | Result
+| Abstract method getNextKeyHint ( KeyValue ) has been removed from this class.|A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method transform ( KeyValue ) has been removed from this class.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+*  HBASE-12296 Filters should work with ByteBufferedCell.
+*  HConnection is removed in HBase 2.0.
+*  RegionLoad and ServerLoad internally moved to shaded PB.
+
+===== [−] class RegionLoad (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Type of field regionLoadPB has been changed from protobuf.generated.ClusterStatusProtos.RegionLoad to shaded.protobuf.generated.ClusterStatusProtos.RegionLoad.|A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+* HBASE-15783:AccessControlConstants#OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST is not used any more.
+package org.apache.hadoop.hbase.security.access
+
+===== [−] interface AccessControlConstants (3)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Field OP_ATTRIBUTE_ACL_STRATEGY of type java.lang.Stringhas been removed from this interface.| A client program may be interrupted by NoSuchFieldError exception.
+| Field OP_ATTRIBUTE_ACL_STRATEGY_CELL_FIRST of type byte[] has been removed from this interface.| A client program may be interrupted by NoSuchFieldError exception.
+| Field OP_ATTRIBUTE_ACL_STRATEGY_DEFAULT of type byte[] has been removed from this interface.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+===== ServerLoad returns long instead of int 1
+hbase-client-1.0.0.jar, ServerLoad.class package org.apache.hadoop.hbase
+
+===== [−] ServerLoad.getNumberOfRequests ( )  :  int  1
+org/apache/hadoop/hbase/ServerLoad.getNumberOfRequests:()I
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from int to long.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] ServerLoad.getReadRequestsCount ( )  :  int  1
+org/apache/hadoop/hbase/ServerLoad.getReadRequestsCount:()I
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from int to long.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−] ServerLoad.getTotalNumberOfRequests ( )  :  int  1
+org/apache/hadoop/hbase/ServerLoad.getTotalNumberOfRequests:()I
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from int to long.|This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+===== [−]ServerLoad.getWriteRequestsCount ( )  :  int  1
+org/apache/hadoop/hbase/ServerLoad.getWriteRequestsCount:()I
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from int to long.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* HBASE-13636 Remove deprecation for HBASE-4072 (Reading of zoo.cfg)
+* HConstants are removed. HBASE-16040 Remove configuration "hbase.replication"
+
+===== [−]class HConstants (6)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Field DEFAULT_HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type boolean has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field HBASE_CONFIG_READ_ZOOKEEPER_CONFIG of type java.lang.String has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field REPLICATION_ENABLE_DEFAULT of type boolean has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field REPLICATION_ENABLE_KEY of type java.lang.String has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field ZOOKEEPER_CONFIG_NAME of type java.lang.String has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+| Field ZOOKEEPER_USEMULTI of type java.lang.String has been removed from this class.| A client program may be interrupted by NoSuchFieldError exception.
+|===
+
+* HBASE-18732: [compat 1-2] HBASE-14047 removed Cell methods without deprecation cycle.
+
+===== [−]interface Cell  5
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method getFamily ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getMvccVersion ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getQualifier ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getRow ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+| Abstract method getValue ( ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* HBASE-18795:Expose KeyValue.getBuffer() for tests alone. Allows KV#getBuffer in tests only that was deprecated previously.
+
+==== Region scanner changes:
+===== [−]interface RegionScanner (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Abstract method boolean nextRaw ( List<Cell>, int ) has been removed from this interface.| A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== StoreFile changes:
+===== [−] class StoreFile (1)
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| This class became interface.| A client program may be interrupted by IncompatibleClassChangeError or InstantiationError exception dependent on the usage of this class.
+|===
+
+==== Mapreduce changes:
+HFile*Format has been removed in HBase 2.0.
+
+==== ClusterStatus changes:
+HBASE-15843: Replace RegionState.getRegionInTransition() Map with a Set
+hbase-client-1.0.0.jar, ClusterStatus.class package org.apache.hadoop.hbase
+
+===== [−] ClusterStatus.getRegionsInTransition ( )  :  Map<String,RegionState>  1
+org/apache/hadoop/hbase/ClusterStatus.getRegionsInTransition:()Ljava/util/Map;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+|Return value type has been changed from java.util.Map<java.lang.String,master.RegionState> to java.util.List<master.RegionState>.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+Other changes in ClusterStatus include removal of convert methods that were no longer necessary after purge of PB from API.
+
+==== Purge of PBs from API
+PBs have been deprecated in APIs in HBase 2.0.
+
+===== [−] HBaseSnapshotException.getSnapshotDescription ( )  :  HBaseProtos.SnapshotDescription  1
+org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.getSnapshotDescription:()Lorg/apache/hadoop/hbase/protobuf/generated/HBaseProtos$SnapshotDescription;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription to org.apache.hadoop.hbase.client.SnapshotDescription.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+* HBASE-15609: Remove PB references from Result, DoubleColumnInterpreter and any such public facing class for 2.0.
+hbase-client-1.0.0.jar, Result.class package org.apache.hadoop.hbase.client
+
+===== [−] Result.getStats ( )  :  ClientProtos.RegionLoadStats  1
+org/apache/hadoop/hbase/client/Result.getStats:()Lorg/apache/hadoop/hbase/protobuf/generated/ClientProtos$RegionLoadStats;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionLoadStats to RegionLoadStats.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== REST changes:
+hbase-rest-1.0.0.jar, Client.class package org.apache.hadoop.hbase.rest.client
+
+===== [−] Client.getHttpClient ( )  :  HttpClient  1
+org/apache/hadoop/hbase/rest/client/Client.getHttpClient:()Lorg/apache/commons/httpclient/HttpClient
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.commons.httpclient.HttpClient to org.apache.http.client.HttpClient.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+hbase-rest-1.0.0.jar, Response.class package org.apache.hadoop.hbase.rest.client
+
+===== [−] Response.getHeaders ( )  :  Header[ ]  1
+org/apache/hadoop/hbase/rest/client/Response.getHeaders:()[Lorg/apache/commons/httpclient/Header;
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from org.apache.commons.httpclient.Header[] to org.apache.http.Header[].| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== PrettyPrinter changes:
+hbase-server-1.0.0.jar, HFilePrettyPrinter.class package org.apache.hadoop.hbase.io.hfile
+
+===== [−]HFilePrettyPrinter.processFile ( Path file )  :  void  1
+org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.processFile:(Lorg/apache/hadoop/fs/Path;)V
+[cols="1,1", frame="all"]
+|===
+| Change | Result
+| Return value type has been changed from void to int.| This method has been removed because the return type is part of the method signature. A client program may be interrupted by NoSuchMethodError exception.
+|===
+
+==== AccessControlClient changes:
+HBASE-13171 Change AccessControlClient methods to accept connection object to reduce setup time. Parameters have been changed in the following methods:
+
+* hbase-client-1.2.7-SNAPSHOT.jar, AccessControlClient.class
+package org.apache.hadoop.hbase.security.access
+AccessControlClient.getUserPermissions ( Configuration conf, String tableRegex ) [static]  :  List<UserPermission> *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.getUserPermissions:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;)Ljava/util/List;
+
+* AccessControlClient.grant ( Configuration conf, String namespace, String userName, Permission.Action... actions )[static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+
+* AccessControlClient.grant ( Configuration conf, String userName, Permission.Action... actions ) [static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+
+* AccessControlClient.grant ( Configuration conf, TableName tableName, String userName, byte[ ] family, byte[ ] qual,Permission.Action... actions ) [static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.grant:(Lorg/apache/hadoop/conf/Configuration;Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;[B[B[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+
+* AccessControlClient.isAccessControllerRunning ( Configuration conf ) [static]  :  boolean *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.isAccessControllerRunning:(Lorg/apache/hadoop/conf/Configuration;)Z
+
+* AccessControlClient.revoke ( Configuration conf, String namespace, String userName, Permission.Action... actions )[static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+
+* AccessControlClient.revoke ( Configuration conf, String userName, Permission.Action... actions ) [static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Ljava/lang/String;[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+
+* AccessControlClient.revoke ( Configuration conf, TableName tableName, String username, byte[ ] family, byte[ ] qualifier,Permission.Action... actions ) [static]  :  void *DEPRECATED*
+org/apache/hadoop/hbase/security/access/AccessControlClient.revoke:(Lorg/apache/hadoop/conf/Configuration;Lorg/apache/hadoop/hbase/TableName;Ljava/lang/String;[B[B[Lorg/apache/hadoop/hbase/security/access/Permission$Action;)V
+* HBASE-18731: [compat 1-2] Mark protected methods of QuotaSettings that touch Protobuf internals as IA.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/src/main/asciidoc/_chapters/architecture.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/architecture.adoc b/src/main/asciidoc/_chapters/architecture.adoc
index 20c5a00..e1905bc 100644
--- a/src/main/asciidoc/_chapters/architecture.adoc
+++ b/src/main/asciidoc/_chapters/architecture.adoc
@@ -594,6 +594,80 @@ See <<regions.arch.assignment>> for more information on region assignment.
 Periodically checks and cleans up the `hbase:meta` table.
 See <<arch.catalog.meta>> for more information on the meta table.
 
+[[master.wal]]
+=== MasterProcWAL
+
+HMaster records administrative operations and their running states, such as the handling of a crashed server,
+table creation, and other DDLs, into its own WAL file. The WALs are stored under the MasterProcWALs
+directory. The Master WALs are not like RegionServer WALs. Keeping up the Master WAL allows
+us run a state machine that is resilient across Master failures. For example, if a HMaster was in the
+middle of creating a table encounters an issue and fails, the next active HMaster can take up where
+the previous left off and carry the operation to completion. Since hbase-2.0.0, a
+new AssignmentManager (A.K.A AMv2) was introduced and the HMaster handles region assignment
+operations, server crash processing, balancing, etc., all via AMv2 persisting all state and
+transitions into MasterProcWALs rather than up into ZooKeeper, as we do in hbase-1.x.
+
+See <<amv2>> (and <<pv2>> for its basis) if you would like to learn more about the new
+AssignmentManager.
+
+[[master.wal.conf]]
+==== Configurations for MasterProcWAL
+Here are the list of configurations that effect MasterProcWAL operation.
+You should not have to change your defaults.
+
+[[hbase.procedure.store.wal.periodic.roll.msec]]
+*`hbase.procedure.store.wal.periodic.roll.msec`*::
++
+.Description
+Frequency of generating a new WAL
++
+.Default
+`1h (3600000 in msec)`
+
+[[hbase.procedure.store.wal.roll.threshold]]
+*`hbase.procedure.store.wal.roll.threshold`*::
++
+.Description
+Threshold in size before the WAL rolls. Every time the WAL reaches this size or the above period, 1 hour, passes since last log roll, the HMaster will generate a new WAL.
++
+.Default
+`32MB (33554432 in byte)`
+
+[[hbase.procedure.store.wal.warn.threshold]]
+*`hbase.procedure.store.wal.warn.threshold`*::
++
+.Description
+If the number of WALs goes beyond this threshold, the following message should appear in the HMaster log with WARN level when rolling.
+
+ procedure WALs count=xx above the warning threshold 64. check running procedures to see if something is stuck.
+
++
+.Default
+`64`
+
+[[hbase.procedure.store.wal.max.retries.before.roll]]
+*`hbase.procedure.store.wal.max.retries.before.roll`*::
++
+.Description
+Max number of retry when syncing slots (records) to its underlying storage, such as HDFS. Every attempt, the following message should appear in the HMaster log.
+
+ unable to sync slots, retry=xx
+
++
+.Default
+`3`
+
+[[hbase.procedure.store.wal.sync.failure.roll.max]]
+*`hbase.procedure.store.wal.sync.failure.roll.max`*::
++
+.Description
+After the above 3 retrials, the log is rolled and the retry count is reset to 0, thereon a new set of retrial starts. This configuration controls the max number of attempts of log rolling upon sync failure. That is, HMaster is allowed to fail to sync 9 times in total. Once it exceeds, the following log should appear in the HMaster log.
+
+ Sync slots after log roll failed, abort.
++
+.Default
+`3`
+
 [[regionserver.arch]]
 == RegionServer
 
@@ -1056,6 +1130,9 @@ Look for the lines like the below in the RegionServer log to see which provider
 2018-04-02 13:22:37,983 INFO  [regionserver/ve0528:16020] wal.WALFactory: Instantiating WALProvider of type class org.apache.hadoop.hbase.wal.AsyncFSWALProvider
 ----
 
+NOTE: As the _AsyncFSWAL_ hacks into the internal of DFSClient implementation, it will be easily broken by upgrading the hadoop dependencies, even for a simple patch release. So if you do not specify the wal provider explicitly, we will first try to use the _asyncfs_, if failed, we will fall back to use _filesystem_. And notice that this may not always work, so if you still have problem starting HBase due to the problem of starting _AsyncFSWAL_, please specify _filesystem_ explicitly in the config file.
+
+NOTE: EC support has been added to hadoop-3.x, and it is incompatible with WAL as the EC output stream does not support hflush/hsync. In order to create a non-EC file in an EC directory, we need to use the new builder-based create API for _FileSystem_, but it is only introduced in hadoop-2.9+ and for HBase we still need to support hadoop-2.7.x. So please do not enable EC for the WAL directory until we find a way to deal with it.
 
 ==== MultiWAL
 With a single WAL per RegionServer, the RegionServer must write to the WAL serially, because HDFS files must be sequential. This causes the WAL to be a performance bottleneck.
@@ -1843,6 +1920,14 @@ See <<managed.compactions>>.
 Compactions do not perform region merges.
 See <<ops.regionmgt.merge>> for more information on region merging.
 
+.Compaction Switch
+We can switch on and off the compactions at region servers. Switching off compactions will also
+interrupt any currently ongoing compactions. It can be done dynamically using the "compaction_switch"
+command from hbase shell. If done from the command line, this setting will be lost on restart of the
+server. To persist the changes across region servers modify the configuration hbase.regionserver
+.compaction.enabled in hbase-site.xml and restart HBase.
+
+
 [[compaction.file.selection]]
 ===== Compaction Policy - HBase 0.96.x and newer
 
@@ -2538,6 +2623,8 @@ See the above HDFS Architecture link for more information.
 [[arch.timelineconsistent.reads]]
 == Timeline-consistent High Available Reads
 
+NOTE: The current <<amv2, Assignment Manager V2>> does not work well with region replica, so this feature maybe broken. Use it with caution.
+
 [[casestudies.timelineconsistent.intro]]
 === Introduction
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/src/main/asciidoc/_chapters/community.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/community.adoc b/src/main/asciidoc/_chapters/community.adoc
index 1a4d977..3a896cf 100644
--- a/src/main/asciidoc/_chapters/community.adoc
+++ b/src/main/asciidoc/_chapters/community.adoc
@@ -94,6 +94,9 @@ NOTE: End-of-life releases are not included in this list.
 | 2.0
 | Michael Stack
 
+| 2.1
+| Duo Zhang
+
 |===
 
 [[hbase.commit.msg.format]]

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/src/main/asciidoc/_chapters/configuration.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/configuration.adoc b/src/main/asciidoc/_chapters/configuration.adoc
index c71e0d1..113058c 100644
--- a/src/main/asciidoc/_chapters/configuration.adoc
+++ b/src/main/asciidoc/_chapters/configuration.adoc
@@ -230,7 +230,9 @@ Use the following legend to interpret this table:
 |Hadoop-2.8.2 | NT | NT | NT | NT | NT
 |Hadoop-2.8.3+ | NT | NT | NT | S | S
 |Hadoop-2.9.0 | X | X | X | X | X
-|Hadoop-3.0.0 | NT | NT | NT | NT | NT
+|Hadoop-2.9.1+ | NT | NT | NT | NT | NT
+|Hadoop-3.0.x | X | X | X | X | X
+|Hadoop-3.1.0 | X | X | X | X | X
 |===
 
 .Hadoop Pre-2.6.1 and JDK 1.8 Kerberos
@@ -257,6 +259,18 @@ data loss. This patch is present in Apache Hadoop releases 2.6.1+.
 Starting around the time of Hadoop version 2.7.0, the Hadoop PMC got into the habit of calling out new minor releases on their major version 2 release line as not stable / production ready. As such, HBase expressly advises downstream users to avoid running on top of these releases. Note that additionally the 2.8.1 release was given the same caveat by the Hadoop PMC. For reference, see the release announcements for link:https://s.apache.org/hadoop-2.7.0-announcement[Apache Hadoop 2.7.0], link:https://s.apache.org/hadoop-2.8.0-announcement[Apache Hadoop 2.8.0], link:https://s.apache.org/hadoop-2.8.1-announcement[Apache Hadoop 2.8.1], and link:https://s.apache.org/hadoop-2.9.0-announcement[Apache Hadoop 2.9.0].
 ====
 
+.Hadoop 3.0.x Releases
+[TIP]
+====
+Hadoop distributions that include the Application Timeline Service feature may cause unexpected versions of HBase classes to be present in the application classpath. Users planning on running MapReduce applications with HBase should make sure that link:https://issues.apache.org/jira/browse/YARN-7190[YARN-7190] is present in their YARN service (currently fixed in 2.9.1+ and 3.1.0+).
+====
+
+.Hadoop 3.1.0 Release
+[TIP]
+====
+The Hadoop PMC called out the 3.1.0 release as not stable / production ready. As such, HBase expressly advises downstream users to avoid running on top of this release. For reference, see the link:https://s.apache.org/hadoop-3.1.0-announcement[release announcement for Hadoop 3.1.0].
+====
+
 .Replace the Hadoop Bundled With HBase!
 [NOTE]
 ====

http://git-wip-us.apache.org/repos/asf/hbase/blob/d49fca13/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 c1299a6..8f65c4e 100644
--- a/src/main/asciidoc/_chapters/external_apis.adoc
+++ b/src/main/asciidoc/_chapters/external_apis.adoc
@@ -851,23 +851,14 @@ println(Bytes.toString(value))
 === 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'}
+the Jython JARs required by your code.
 
-/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.
+Set the path to directory containing the `jython.jar` and each additional Jython-related JAR needed for
+your project. Then export HBASE_CLASSPATH pointing to the $JYTHON_HOME env. variable.
 
 [source, bash]
 ----
-$ export CLASSPATH=$CLASSPATH:/usr/lib/hadoop/lib/native:/usr/lib/hbase/lib/native/Linux-amd64-64:/path/to/jython.jar
+$ export HBASE_CLASSPATH=/directory/jython.jar
 ----
 
 Start a Jython shell with HBase and Hadoop JARs in the classpath:
@@ -877,55 +868,52 @@ $ bin/hbase org.python.util.jython
 
 .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.
+The following Jython code example checks for table,
+if it exists, deletes it and then creates it. Then it
+populates the table with data and fetches the data.
 
 [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
+from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, TableName
+from org.apache.hadoop.hbase.client import Admin, Connection, ConnectionFactory, Get, Put, Result, Table
+from org.apache.hadoop.conf import Configuration
 
 # 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()
+conf = HBaseConfiguration.create()
+connection = ConnectionFactory.createConnection(conf)
+admin = connection.getAdmin()
 
-# 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")
+# Create a table named 'test' that has a column family
+# named 'content'.
+tableName = TableName.valueOf("test")
+table = connection.getTable(tableName)
 
-desc = HTableDescriptor(tablename)
-desc.addFamily(HColumnDescriptor("content:"))
-desc.addFamily(HColumnDescriptor("anchor:"))
-admin = HBaseAdmin(conf)
+desc = HTableDescriptor(tableName)
+desc.addFamily(HColumnDescriptor("content"))
 
 # Drop and recreate if it exists
-if admin.tableExists(tablename):
-    admin.disableTable(tablename)
-    admin.deleteTable(tablename)
-admin.createTable(desc)
+if admin.tableExists(tableName):
+    admin.disableTable(tableName)
+    admin.deleteTable(tableName)
 
-tables = admin.listTables()
-table = HTable(conf, tablename)
+admin.createTable(desc)
 
 # Add content to 'column:' on a row named 'row_x'
 row = 'row_x'
-update = Get(row)
-update.put('content:', 'some content')
-table.commit(update)
+put = Put(row)
+put.addColumn("content", "qual", "some content")
+table.put(put)
 
 # Now fetch the content just added, returns a byte[]
-data_row = table.get(row, "content:")
-data = java.lang.String(data_row.value, "UTF8")
+get = Get(row)
 
-print "The fetched row contains the value '%s'" % data
+result = table.get(get)
+data = java.lang.String(result.getValue("content", "qual"), "UTF8")
 
-# Delete the table.
-admin.disableTable(desc.getName())
-admin.deleteTable(desc.getName())
+print "The fetched row contains the value '%s'" % data
 ----
 ====
 
@@ -935,24 +923,23 @@ This example scans a table and returns the results that match a given family qua
 
 [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], "")
+from org.apache.hadoop.hbase import TableName, HBaseConfiguration
+from org.apache.hadoop.hbase.client import Connection, ConnectionFactory, Result, ResultScanner, Table, Admin
+from org.apache.hadoop.conf import Configuration
+conf = HBaseConfiguration.create()
+connection = ConnectionFactory.createConnection(conf)
+admin = connection.getAdmin()
+tableName = TableName.valueOf('wiki')
+table = connection.getTable(tableName)
+
+cf = "title"
+attr = "attr"
+scanner = table.getScanner(cf)
 while 1:
     result = scanner.next()
     if not result:
-        break
-    print java.lang.String(result.row), java.lang.String(result.get('title:').value)
+       break
+    print java.lang.String(result.row), java.lang.String(result.getValue(cf, attr))
 ----
 ====