You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/08/09 05:51:17 UTC

[01/27] hbase git commit: HBASE-16893 Use Collection.removeIf instead of Iterator.remove in DependentColumnFilter [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-18467 eec17ac01 -> 26eb5a2c2 (forced update)


HBASE-16893 Use Collection.removeIf instead of Iterator.remove in DependentColumnFilter

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-18467
Commit: 855dd48f0a65e7db7263c076d7ed078bf1295ec5
Parents: 504a1f1
Author: Robert Yokota <ra...@gmail.com>
Authored: Thu Aug 3 13:37:45 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Aug 3 13:38:52 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/filter/DependentColumnFilter.java | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/855dd48f/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
index 5717fc4..90d00dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
@@ -153,14 +152,7 @@ public class DependentColumnFilter extends CompareFilter {
 
   @Override
   public void filterRowCells(List<Cell> kvs) {
-    Iterator<? extends Cell> it = kvs.iterator();
-    Cell kv;
-    while(it.hasNext()) {
-      kv = it.next();
-      if(!stampSet.contains(kv.getTimestamp())) {
-        it.remove();
-      }
-    }
+    kvs.removeIf(kv -> !stampSet.contains(kv.getTimestamp()));
   }
 
   @Override


[25/27] hbase git commit: HBASE-18492 [AMv2] Embed code for selecting highest versioned region server for system table regions in AssignmentManager.processAssignQueue()

Posted by bu...@apache.org.
HBASE-18492 [AMv2] Embed code for selecting highest versioned region server for system table regions in AssignmentManager.processAssignQueue()

* Modified AssignmentManager.processAssignQueue() method to consider only highest versioned region servers for system table regions when
  destination server is not specified for them. Destination server is retained, if specified.
* Modified MoveRegionProcedure to allow null value for destination server i.e. moving a region from specific source server to non-specific/ unknown
  destination server (picked by load-balancer) is supported now.
* Removed destination server selection from HMaster.checkIfShouldMoveSystemRegionAsync(), as destination server will be picked by load balancer

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: f314b5911bf89a7de729b5850df0c6f57a735ce0
Parents: 0339068
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Sat Jul 15 22:51:05 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 8 14:02:11 2017 -0700

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     |  3 +-
 .../master/assignment/AssignmentManager.java    | 44 +++++++++++++++-----
 .../master/assignment/MoveRegionProcedure.java  | 15 ++++---
 3 files changed, 46 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f314b591/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 74ae16d..70753c6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -387,7 +387,8 @@ enum MoveRegionState {
 message MoveRegionStateData {
   optional RegionInfo region_info = 1;
   required ServerName source_server = 2;
-  required ServerName destination_server = 3;
+  // if destination server not specified, its selected with load balancer
+  optional ServerName destination_server = 3;
 }
 
 enum GCRegionState {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f314b591/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 255ea5e..54cb1ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -478,15 +478,13 @@ public class AssignmentManager implements ServerListener {
     new Thread(() -> {
       try {
         synchronized (checkIfShouldMoveSystemRegionLock) {
-          List<ServerName> serverList = master.getServerManager()
-              .createDestinationServersList(getExcludedServersForSystemTable());
           List<RegionPlan> plans = new ArrayList<>();
           for (ServerName server : getExcludedServersForSystemTable()) {
             List<HRegionInfo> regionsShouldMove = getCarryingSystemTables(server);
             if (!regionsShouldMove.isEmpty()) {
               for (HRegionInfo regionInfo : regionsShouldMove) {
-                RegionPlan plan = new RegionPlan(regionInfo, server,
-                    getBalancer().randomAssignment(regionInfo, serverList));
+                // null value for dest forces destination server to be selected by balancer
+                RegionPlan plan = new RegionPlan(regionInfo, server, null);
                 if (regionInfo.isMetaRegion()) {
                   // Must move meta region first.
                   moveAsync(plan);
@@ -1648,9 +1646,14 @@ public class AssignmentManager implements ServerListener {
     }
 
     // TODO: Optimize balancer. pass a RegionPlan?
-    final HashMap<HRegionInfo, ServerName> retainMap = new HashMap<HRegionInfo, ServerName>();
-    final List<HRegionInfo> rrList = new ArrayList<HRegionInfo>();
-    for (RegionStateNode regionNode: regions.values()) {
+    final HashMap<HRegionInfo, ServerName> retainMap = new HashMap<>();
+    final List<HRegionInfo> userRRList = new ArrayList<>();
+    // regions for system tables requiring reassignment
+    final List<HRegionInfo> sysRRList = new ArrayList<>();
+    for (RegionStateNode regionNode : regions.values()) {
+      boolean sysTable = regionNode.isSystemTable();
+      final List<HRegionInfo> rrList = sysTable ? sysRRList : userRRList;
+
       if (regionNode.getRegionLocation() != null) {
         retainMap.put(regionNode.getRegionInfo(), regionNode.getRegionLocation());
       } else {
@@ -1659,7 +1662,6 @@ public class AssignmentManager implements ServerListener {
     }
 
     // TODO: connect with the listener to invalidate the cache
-    final LoadBalancer balancer = getBalancer();
 
     // TODO use events
     List<ServerName> servers = master.getServerManager().createDestinationServersList();
@@ -1679,13 +1681,35 @@ public class AssignmentManager implements ServerListener {
       servers = master.getServerManager().createDestinationServersList();
     }
 
-    final boolean isTraceEnabled = LOG.isTraceEnabled();
+    if (!sysRRList.isEmpty()) {
+      // system table regions requiring reassignment are present, get region servers
+      // not available for system table regions
+      final List<ServerName> excludeServers = getExcludedServersForSystemTable();
+      List<ServerName> serversForSysTables = servers.stream()
+          .filter(s -> !excludeServers.contains(s)).collect(Collectors.toList());
+      if (serversForSysTables.isEmpty()) {
+        LOG.warn("No servers available for system table regions, considering all servers!");
+      }
+      LOG.debug("Processing assignment plans for System tables sysServersCount=" +
+          serversForSysTables.size() + ", allServersCount=" + servers.size());
+      processAssignmentPlans(regions, null, sysRRList,
+          serversForSysTables.isEmpty() ? servers : serversForSysTables);
+    }
+
+    processAssignmentPlans(regions, retainMap, userRRList, servers);
+  }
+
+  private void processAssignmentPlans(final HashMap<HRegionInfo, RegionStateNode> regions,
+      final HashMap<HRegionInfo, ServerName> retainMap, final List<HRegionInfo> rrList,
+      final List<ServerName> servers) {
+    boolean isTraceEnabled = LOG.isTraceEnabled();
     if (isTraceEnabled) {
       LOG.trace("available servers count=" + servers.size() + ": " + servers);
     }
 
+    final LoadBalancer balancer = getBalancer();
     // ask the balancer where to place regions
-    if (!retainMap.isEmpty()) {
+    if (retainMap != null && !retainMap.isEmpty()) {
       if (isTraceEnabled) {
         LOG.trace("retain assign regions=" + retainMap);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f314b591/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
index d8c1b7d..1907e98 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
@@ -54,7 +54,6 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
 
   public MoveRegionProcedure(final MasterProcedureEnv env, final RegionPlan plan) {
     super(env, plan.getRegionInfo());
-    assert plan.getDestination() != null: plan.toString();
     this.plan = plan;
   }
 
@@ -70,7 +69,10 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
         setNextState(MoveRegionState.MOVE_REGION_ASSIGN);
         break;
       case MOVE_REGION_ASSIGN:
-        addChildProcedure(new AssignProcedure(plan.getRegionInfo(), plan.getDestination()));
+        AssignProcedure assignProcedure = plan.getDestination() == null ?
+            new AssignProcedure(plan.getRegionInfo(), true) :
+            new AssignProcedure(plan.getRegionInfo(), plan.getDestination());
+        addChildProcedure(assignProcedure);
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -127,8 +129,10 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
 
     final MoveRegionStateData.Builder state = MoveRegionStateData.newBuilder()
         // No need to serialize the HRegionInfo. The super class has the region.
-        .setSourceServer(ProtobufUtil.toServerName(plan.getSource()))
-        .setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
+        .setSourceServer(ProtobufUtil.toServerName(plan.getSource()));
+    if (plan.getDestination() != null) {
+      state.setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
+    }
     state.build().writeDelimitedTo(stream);
   }
 
@@ -139,7 +143,8 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
     final MoveRegionStateData state = MoveRegionStateData.parseDelimitedFrom(stream);
     final HRegionInfo regionInfo = getRegion(); // Get it from super class deserialization.
     final ServerName sourceServer = ProtobufUtil.toServerName(state.getSourceServer());
-    final ServerName destinationServer = ProtobufUtil.toServerName(state.getDestinationServer());
+    final ServerName destinationServer = state.hasDestinationServer() ?
+        ProtobufUtil.toServerName(state.getDestinationServer()) : null;
     this.plan = new RegionPlan(regionInfo, sourceServer, destinationServer);
   }
 }
\ No newline at end of file

[15/27] hbase git commit: HBASE-14220 nightly check that we can build a source tarball.

Posted by bu...@apache.org.
HBASE-14220 nightly check that we can build a source tarball.

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 2a717459bdad0f8ab64ecc45300e9022cf210af9
Parents: 04424ad
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Jul 31 01:08:16 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Sun Aug 6 12:01:47 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 56 ++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 56 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2a717459/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index b5c7660..94d8c3f 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -241,6 +241,62 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
         }
       }
     }
+    // This is meant to mimic what a release manager will do to create RCs.
+    // See http://hbase.apache.org/book.html#maven.release
+    stage ('create source tarball') {
+      tools {
+        maven 'Maven (latest)'
+        // this needs to be set to the jdk that ought to be used to build releases on the branch the Jenkinsfile is stored in.
+        jdk "JDK 1.8 (latest)"
+      }
+      steps {
+        sh 'mvn --version --offline'
+        dir ("component") {
+          sh '''#!/bin/bash -e
+            echo "Clean..."
+            mvn --batch-mode -DskipTests clean
+            echo "Step 3 Build the source tarball"
+            mvn -Prelease --batch-mode -Dmaven.repo.local="${WORKSPACE}/.m2-for-repo" install -DskipTests assembly:single -Dassembly.file=hbase-assembly/src/main/assembly/src.xml
+'''
+        }
+        dir ("unpacked_src_tarball") {
+          sh '''#!/bin/bash -e
+            echo "Unpack the source tarball"
+            tar --strip-components=1 -xzf ../component/hbase-assembly/target/hbase-*-src.tar.gz
+            echo "Building from source artifact."
+            mvn -DskipTests -Prelease --batch-mode -Dmaven.repo.local="${WORKSPACE}/.m2-for-src" clean install assembly:single
+            echo "Clean up after checking ability to build."
+            mvn -DskipTests --batch-mode clean
+'''
+        }
+        dir ("component") {
+          sh '''#!/bin/bash -e
+            echo "Clean up the source checkout"
+            mvn -DskipTests --batch-mode clean
+            echo "Diff against source tree"
+            diff --binary --recursive . ../unpacked_src_tarball >../diff_output || true
+'''
+        }
+        // expectation check largely based on HBASE-14952
+        sh '''#!/bin/bash -e
+          echo "Checking against things we don't expect to include in the source tarball (git related, hbase-native-client, etc.)"
+          cat >known_excluded <<END
+Only in .: .git
+Only in .: .gitattributes
+Only in .: .gitignore
+Only in .: hbase-native-client
+END
+          if ! diff known_excluded diff_output >/dev/null ; then
+            echo "Any output here are unexpected differences between the source artifact we'd make for an RC and the current branch."
+            echo "The expected differences are on the < side and the current differences are on the > side."
+            echo "In a given set of differences, '.' refers to the branch in the repo and 'unpacked_src_tarball' refers to what we pulled out of the tarball."
+            diff known_excluded diff_output
+          else
+            echo "Everything looks as expected."
+          fi
+'''
+      }
+    }
   }
   post {
     failure {

[03/27] hbase git commit: Add Mike Drob to pom.xml

Posted by bu...@apache.org.
Add Mike Drob to pom.xml


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

Branch: refs/heads/HBASE-18467
Commit: 6266bb352b99df76b581c9af2bb4c185127e7da7
Parents: fe890b7
Author: Mike Drob <md...@apache.org>
Authored: Thu Aug 3 15:51:34 2017 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Thu Aug 3 15:51:34 2017 -0500

----------------------------------------------------------------------
 pom.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6266bb35/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 176f362..f3b3f36 100644
--- a/pom.xml
+++ b/pom.xml
@@ -411,6 +411,12 @@
       <timezone>0</timezone>
     </developer>
     <developer>
+      <id>mdrob</id>
+      <name>Mike Drob</name>
+      <email>mdrob@apache.org</email>
+      <timezone>-5</timezone>
+    </developer>
+    <developer>
       <id>misty</id>
       <name>Misty Stanley-Jones</name>
       <email>misty@apache.org</email>

[17/27] hbase git commit: HBASE-18515 Introduce Delete.add as a replacement for Delete#addDeleteMarker

Posted by bu...@apache.org.
HBASE-18515 Introduce Delete.add as a replacement for Delete#addDeleteMarker

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-18467
Commit: a7014ce46c15ad5293cc5b124df3463b3c5fa62d
Parents: 5915d73
Author: no_apologies <xy...@163.com>
Authored: Mon Aug 7 11:04:31 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Aug 7 11:05:23 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Delete.java    | 18 ++++++++++++++----
 .../hadoop/hbase/protobuf/ProtobufUtil.java       |  2 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java       |  2 +-
 .../coprocessor/TestRowProcessorEndpoint.java     |  2 +-
 .../org/apache/hadoop/hbase/mapreduce/Import.java |  4 ++--
 .../apache/hadoop/hbase/mapreduce/WALPlayer.java  |  2 +-
 .../replication/regionserver/ReplicationSink.java |  2 +-
 .../security/visibility/VisibilityController.java |  2 +-
 .../org/apache/hadoop/hbase/wal/WALSplitter.java  |  2 +-
 9 files changed, 23 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 351d8a6..395c277 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -151,15 +151,26 @@ public class Delete extends Mutation implements Comparable<Row> {
   }
 
   /**
-   * Advanced use only.
-   * Add an existing delete marker to this Delete object.
+   * Advanced use only. Add an existing delete marker to this Delete object.
    * @param kv An existing KeyValue of type "delete".
    * @return this for invocation chaining
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use {@link #add(Cell)}
+   *             instead
    */
   @SuppressWarnings("unchecked")
+  @Deprecated
   public Delete addDeleteMarker(Cell kv) throws IOException {
-    // TODO: Deprecate and rename 'add' so it matches how we add KVs to Puts.
+    return this.add(kv);
+  }
+
+  /**
+   * Add an existing delete marker to this Delete object.
+   * @param kv An existing KeyValue of type "delete".
+   * @return this for invocation chaining
+   * @throws IOException
+   */
+  public Delete add(Cell kv) throws IOException {
     if (!CellUtil.isDelete(kv)) {
       throw new IOException("The recently added KeyValue is not of type "
           + "delete. Rowkey: " + Bytes.toStringBinary(this.row));
@@ -178,7 +189,6 @@ public class Delete extends Mutation implements Comparable<Row> {
     return this;
   }
 
-
   /**
    * Delete all versions of all columns of the specified family.
    * <p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index fcf2c34..01ba0e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -579,7 +579,7 @@ public final class ProtobufUtil {
           delete =
             new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
         }
-        delete.addDeleteMarker(cell);
+        delete.add(cell);
       }
     } else {
       if (delete == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 542ade9..65e95b6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -730,7 +730,7 @@ public final class ProtobufUtil {
           delete =
             new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
         }
-        delete.addDeleteMarker(cell);
+        delete.add(cell);
       }
     } else {
       if (delete == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
index ed53027..39109f8 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
@@ -548,7 +548,7 @@ public class TestRowProcessorEndpoint {
             KeyValue kvDelete =
                 new KeyValue(rows[i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv), 
                     kv.getTimestamp(), KeyValue.Type.Delete);
-            d.addDeleteMarker(kvDelete);
+            d.add(kvDelete);
             Put p = new Put(rows[1 - i]);
             KeyValue kvAdd =
                 new KeyValue(rows[1 - i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 1bb3d25..b5bb2ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -354,7 +354,7 @@ public class Import extends Configured implements Tool {
          */
         if (CellUtil.isDeleteFamily(kv)) {
           Delete deleteFamily = new Delete(key.get());
-          deleteFamily.addDeleteMarker(kv);
+          deleteFamily.add(kv);
           if (durability != null) {
             deleteFamily.setDurability(durability);
           }
@@ -364,7 +364,7 @@ public class Import extends Configured implements Tool {
           if (delete == null) {
             delete = new Delete(key.get());
           }
-          delete.addDeleteMarker(kv);
+          delete.add(kv);
         } else {
           if (put == null) {
             put = new Put(key.get());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index f077659..b1e655c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -182,7 +182,7 @@ public class WALPlayer extends Configured implements Tool {
                 }
               }
               if (CellUtil.isDelete(cell)) {
-                del.addDeleteMarker(cell);
+                del.add(cell);
               } else {
                 put.add(cell);
               }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
index a3d6d13..620b28b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java
@@ -191,7 +191,7 @@ public class ReplicationSink {
               addToHashMultiMap(rowMap, table, clusterIds, m);
             }
             if (CellUtil.isDelete(cell)) {
-              ((Delete) m).addDeleteMarker(cell);
+              ((Delete) m).add(cell);
             } else {
               ((Put) m).add(cell);
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 529ae0b..c6091c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -400,7 +400,7 @@ public class VisibilityController implements MasterObserver, RegionObserver,
                 p.add(cell);
               } else if (m instanceof Delete) {
                 Delete d = (Delete) m;
-                d.addDeleteMarker(cell);
+                d.add(cell);
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7014ce4/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index df4c1bd..4a9741a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -2376,7 +2376,7 @@ public class WALSplitter {
         }
       }
       if (CellUtil.isDelete(cell)) {
-        ((Delete) m).addDeleteMarker(cell);
+        ((Delete) m).add(cell);
       } else {
         ((Put) m).add(cell);
       }


[21/27] hbase git commit: HBASE-18315 Eliminate the findbugs warnings for hbase-rest

Posted by bu...@apache.org.
HBASE-18315 Eliminate the findbugs warnings for hbase-rest


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

Branch: refs/heads/HBASE-18467
Commit: 4376233e2a7456da752ed1a88ad091163b7dc536
Parents: 7e7461e
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Tue Aug 8 10:47:36 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Tue Aug 8 10:47:36 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/rest/RESTServer.java    |  3 ---
 .../hadoop/hbase/rest/model/NamespacesInstanceModel.java | 11 ++++++-----
 2 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4376233e/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
index 5c43287..eb3fc26 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
@@ -125,9 +125,6 @@ public class RESTServer implements Constants {
   static void addCSRFFilter(ServletContextHandler ctxHandler, Configuration conf) {
     restCSRFEnabled = conf.getBoolean(REST_CSRF_ENABLED_KEY, REST_CSRF_ENABLED_DEFAULT);
     if (restCSRFEnabled) {
-      Set<String> restCsrfMethodsToIgnore = new HashSet<>();
-      restCsrfMethodsToIgnore.addAll(getTrimmedStringList(conf,
-          REST_CSRF_METHODS_TO_IGNORE_KEY, REST_CSRF_METHODS_TO_IGNORE_DEFAULT));
       Map<String, String> restCsrfParams = RestCsrfPreventionFilter
           .getFilterParams(conf, "hbase.rest-csrf.");
       FilterHolder holder = new FilterHolder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4376233e/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
index bcc1581..b18fd63 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
@@ -129,11 +129,11 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan
     sb.append(namespaceName);
     sb.append("\'");
     if(properties != null){
-      for(String key: properties.keySet()){
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
         sb.append(", ");
-        sb.append(key);
+        sb.append(entry.getKey());
         sb.append(" => '");
-        sb.append(properties.get(key));
+        sb.append(entry.getValue());
         sb.append("\'");
       }
     }
@@ -145,10 +145,11 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan
   public byte[] createProtobufOutput() {
     NamespaceProperties.Builder builder = NamespaceProperties.newBuilder();
     if(properties != null){
-      for(String key: properties.keySet()){
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
+        String key = entry.getKey();
         NamespaceProperties.Property.Builder property = NamespaceProperties.Property.newBuilder();
         property.setKey(key);
-        property.setValue(properties.get(key));
+        property.setValue(entry.getValue());
         builder.addProps(property);
       }
     }


[13/27] hbase git commit: HBASE-18520 Add jmx value to determine true Master Start time

Posted by bu...@apache.org.
HBASE-18520 Add jmx value to determine true Master Start time

This is to determine how long it took in total for the master to start and finish initializing.

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-18467
Commit: 637f7abf0b3ab33b105346c6627dc9b2170eab53
Parents: 7da4750
Author: Zach York <zy...@amazon.com>
Authored: Thu Jun 15 17:10:07 2017 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Sat Aug 5 22:32:33 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/MetricsMasterSource.java  |  2 ++
 .../apache/hadoop/hbase/master/MetricsMasterWrapper.java |  5 +++++
 .../hadoop/hbase/master/MetricsMasterSourceImpl.java     |  2 ++
 .../java/org/apache/hadoop/hbase/master/HMaster.java     | 11 +++++++++++
 .../hadoop/hbase/master/MetricsMasterWrapperImpl.java    |  5 +++++
 .../apache/hadoop/hbase/regionserver/HRegionServer.java  |  2 +-
 6 files changed, 26 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
index 9163511..d187bb1 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java
@@ -49,6 +49,7 @@ public interface MetricsMasterSource extends BaseSource {
   // Strings used for exporting to metrics system.
   String MASTER_ACTIVE_TIME_NAME = "masterActiveTime";
   String MASTER_START_TIME_NAME = "masterStartTime";
+  String MASTER_FINISHED_INITIALIZATION_TIME_NAME = "masterFinishedInitializationTime";
   String AVERAGE_LOAD_NAME = "averageLoad";
   String LIVE_REGION_SERVERS_NAME = "liveRegionServers";
   String DEAD_REGION_SERVERS_NAME = "deadRegionServers";
@@ -64,6 +65,7 @@ public interface MetricsMasterSource extends BaseSource {
   String CLUSTER_REQUESTS_NAME = "clusterRequests";
   String MASTER_ACTIVE_TIME_DESC = "Master Active Time";
   String MASTER_START_TIME_DESC = "Master Start Time";
+  String MASTER_FINISHED_INITIALIZATION_TIME_DESC = "Timestamp when Master has finished initializing";
   String AVERAGE_LOAD_DESC = "AverageLoad";
   String LIVE_REGION_SERVERS_DESC = "Names of live RegionServers";
   String NUMBER_OF_REGION_SERVERS_DESC = "Number of RegionServers";

http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
index 4789283..80e0892 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapper.java
@@ -134,4 +134,9 @@ public interface MetricsMasterWrapper {
    * Gets the space usage and limit for each namespace.
    */
   Map<String,Entry<Long,Long>> getNamespaceSpaceUtilization();
+
+  /**
+   * Get the time in Millis when the master finished initializing/becoming the active master
+   */
+  long getMasterInitializationTime();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
index f21e29e..51f8fad 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java
@@ -92,6 +92,8 @@ public class MetricsMasterSourceImpl
               MASTER_ACTIVE_TIME_DESC), masterWrapper.getActiveTime())
           .addGauge(Interns.info(MASTER_START_TIME_NAME,
               MASTER_START_TIME_DESC), masterWrapper.getStartTime())
+          .addGauge(Interns.info(MASTER_FINISHED_INITIALIZATION_TIME_NAME, MASTER_FINISHED_INITIALIZATION_TIME_DESC),
+              masterWrapper.getMasterInitializationTime())
           .addGauge(Interns.info(AVERAGE_LOAD_NAME, AVERAGE_LOAD_DESC),
               masterWrapper.getAverageLoad())
           .tag(Interns.info(LIVE_REGION_SERVERS_NAME, LIVE_REGION_SERVERS_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5316b54..d6b149a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -371,6 +371,9 @@ public class HMaster extends HRegionServer implements MasterServices {
   // Time stamps for when a hmaster became active
   private long masterActiveTime;
 
+  // Time stamp for when HMaster finishes becoming Active Master
+  private long masterFinishedInitializationTime;
+
   //should we check the compression codec type at master side, default true, HBASE-6370
   private final boolean masterCheckCompression;
 
@@ -875,6 +878,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     status.markComplete("Initialization successful");
     LOG.info(String.format("Master has completed initialization %.3fsec",
        (System.currentTimeMillis() - masterActiveTime) / 1000.0f));
+    this.masterFinishedInitializationTime = System.currentTimeMillis();
     configurationManager.registerObserver(this.balancer);
     configurationManager.registerObserver(this.hfileCleaner);
 
@@ -2513,6 +2517,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     return masterActiveTime;
   }
 
+  /**
+   * @return timestamp in millis when HMaster finished becoming the active master
+   */
+  public long getMasterFinishedInitializationTime() {
+    return masterFinishedInitializationTime;
+  }
+
   public int getNumWALFiles() {
     return procedureStore != null ? procedureStore.getActiveLogs().size() : 0;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
index ed37d19..518c7b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
@@ -59,6 +59,11 @@ public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
   }
 
   @Override
+  public long getMasterInitializationTime() {
+    return master.getMasterFinishedInitializationTime();
+  }
+
+  @Override
   public String getClusterId() {
     return master.getClusterId();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/637f7abf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 184f070..0774df1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -545,6 +545,7 @@ public class HRegionServer extends HasThread implements
    */
   public HRegionServer(Configuration conf, CoordinatedStateManager csm) throws IOException {
     super("RegionServer");  // thread name
+    this.startcode = System.currentTimeMillis();
     this.fsOk = true;
     this.conf = conf;
     // initialize netty event loop group at the very beginning as we may use it to start rpc server,
@@ -590,7 +591,6 @@ public class HRegionServer extends HasThread implements
     this.stopped = false;
 
     rpcServices = createRpcServices();
-    this.startcode = System.currentTimeMillis();
     if (this instanceof HMaster) {
       useThisHostnameInstead = conf.get(MASTER_HOSTNAME_KEY);
     } else {

[27/27] hbase git commit: HBASE-18467 WIP fail yetus checks instead of running.

Posted by bu...@apache.org.
HBASE-18467 WIP fail yetus checks instead of running.


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

Branch: refs/heads/HBASE-18467
Commit: 26eb5a2c2cba4ac67da5229319cd537fca540709
Parents: 436d99d
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Aug 9 00:50:26 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Aug 9 00:50:26 2017 -0500

----------------------------------------------------------------------
 dev-support/hbase_nightly_yetus.sh | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/26eb5a2c/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index 007d64a..a33cbd4 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -16,6 +16,9 @@
 # specific language governing permissions and limitations
 # under the License.
 
+# fail fast
+exit 1772
+
 declare -i missing_env=0
 # Validate params
 for required_env in "TESTS" "TOOLS" "BASEDIR" "ARCHIVE_PATTERN_LIST" "OUTPUT_RELATIVE" \

[11/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/custom/project-info-report.properties
----------------------------------------------------------------------
diff --git a/src/main/site/custom/project-info-report.properties b/src/main/site/custom/project-info-report.properties
deleted file mode 100644
index 912339e..0000000
--- a/src/main/site/custom/project-info-report.properties
+++ /dev/null
@@ -1,303 +0,0 @@
-# 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.
-
-report.cim.access                                                  = Access
-report.cim.anthill.intro                                           = Apache HBase&#8482; uses {Anthill, http://www.anthillpro.com/html/products/anthillos/}.
-report.cim.bamboo.intro                                            = Apache HBase&#8482; uses {Bamboo, http://www.atlassian.com/software/bamboo/}.
-report.cim.buildforge.intro                                        = Apache HBase&#8482; uses {Build Forge, http://www-306.ibm.com/software/awdtools/buildforge/enterprise/}.
-report.cim.continuum.intro                                         = Apache HBase&#8482; uses {Continuum, http://continuum.apache.org/}.
-report.cim.cruisecontrol.intro                                     = Apache HBase&#8482; uses {CruiseControl, http://cruisecontrol.sourceforge.net/}.
-report.cim.description                                             = These are the definitions of all continuous integration processes that builds and tests code on a frequent, regular basis.
-report.cim.general.intro                                           = Apache HBase&#8482; uses Continuous Integration System.
-report.cim.hudson.intro                                            = Apache HBase&#8482; uses {Hudson, http://hudson-ci.org/}.
-report.cim.jenkins.intro                                           = Apache HBase&#8482; uses {Jenkins, http://jenkins-ci.org/}.
-report.cim.luntbuild.intro                                         = Apache HBase&#8482; uses {Luntbuild, http://luntbuild.javaforge.com/}.
-report.cim.travis.intro                                            = Apache HBase&#8482; uses {Travis CI, https://travis-ci.org/}.
-report.cim.name                                                    = Continuous Integration
-report.cim.nocim                                                   = No continuous integration management system is defined. Please check back at a later date.
-report.cim.notifiers.column.address                                = Address
-report.cim.notifiers.column.configuration                          = Configuration
-report.cim.notifiers.column.type                                   = Type
-report.cim.notifiers.intro                                         = Configuration for notifying developers/users when a build is unsuccessful, including user information and notification mode.
-report.cim.notifiers.nolist                                        = No notifiers are defined. Please check back at a later date.
-report.cim.notifiers.title                                         = Notifiers
-report.cim.nourl                                                   = No url to the continuous integration system is defined.
-report.cim.overview.title                                          = Overview
-report.cim.title                                                   = Continuous Integration
-report.cim.url                                                     = This is a link to the continuous integration system used by the project:
-report.dependencies.column.artifactId                              = ArtifactId
-report.dependencies.column.classifier                              = Classifier
-report.dependencies.column.description                             = Description
-report.dependencies.column.groupId                                 = GroupId
-report.dependencies.column.license                                 = License
-report.dependencies.column.optional                                = Optional
-report.dependencies.column.isOptional                              = Yes
-report.dependencies.column.isNotOptional                           = No
-report.dependencies.column.type                                    = Type
-report.dependencies.column.url                                     = URL
-report.dependencies.column.version                                 = Version
-report.dependencies.description                                    = This is a list of project's dependencies and provides information on each dependency.
-report.dependencies.file.details.cell.debuginformation.yes         = Yes
-report.dependencies.file.details.cell.debuginformation.no          = No
-report.dependencies.file.details.column.classes                    = Classes
-report.dependencies.file.details.column.debuginformation           = Debug Information
-report.dependencies.file.details.column.entries                    = Entries
-report.dependencies.file.details.column.file                       = Filename
-report.dependencies.file.details.column.javaVersion                = Java Version
-report.dependencies.file.details.column.packages                   = Packages
-report.dependencies.file.details.column.sealed                     = Sealed
-report.dependencies.file.details.column.size                       = Size
-report.dependencies.file.details.column.size.gb                    = GB
-report.dependencies.file.details.column.size.mb                    = MB
-report.dependencies.file.details.column.size.kb                    = kB
-report.dependencies.file.details.columntitle.debuginformation      = Indicates whether these dependencies have been compiled with debug information.
-report.dependencies.file.details.title                             = Dependency File Details
-report.dependencies.file.details.total                             = Total
-report.dependencies.graph.tables.licenses                          = Licenses
-report.dependencies.graph.tables.unknown                           = Unknown
-report.dependencies.graph.title                                    = Apache HBase&#8482; Dependency Graph
-report.dependencies.graph.tree.title                               = Dependency Tree
-report.dependencies.intro.compile                                  = This is a list of compile dependencies for Apache HBase&#8482;. These dependencies are required to compile and run the application:
-report.dependencies.intro.provided                                 = This is a list of provided dependencies for Apache HBase&#8482;. These dependencies are required to compile the application, but should be provided by default when using the library:
-report.dependencies.intro.runtime                                  = This is a list of runtime dependencies for Apache HBase&#8482;. These dependencies are required to run the application:
-report.dependencies.intro.system                                   = This is a list of system dependencies for Apache HBase&#8482;. These dependencies are required to compile the application:
-report.dependencies.intro.test                                     = This is a list of test dependencies for Apache HBase&#8482;. These dependencies are only required to compile and run unit tests for the application:
-report.dependencies.name                                           = Dependencies
-report.dependencies.nolist                                         = There are no dependencies for Apache HBase&#8482;. It is a standalone application that does not depend on any other project.
-report.dependencies.repo.locations.artifact.breakdown              = Repository locations for each of the Dependencies.
-report.dependencies.repo.locations.cell.release.disabled           = No
-report.dependencies.repo.locations.cell.release.enabled            = Yes
-report.dependencies.repo.locations.cell.snapshot.disabled          = No
-report.dependencies.repo.locations.cell.snapshot.enabled           = Yes
-report.dependencies.repo.locations.cell.blacklisted.disabled       = No
-report.dependencies.repo.locations.cell.blacklisted.enabled        = Yes
-report.dependencies.repo.locations.column.artifact                 = Artifact
-report.dependencies.repo.locations.column.blacklisted              = Blacklisted
-report.dependencies.repo.locations.column.release                  = Release
-report.dependencies.repo.locations.column.repoid                   = Repo ID
-report.dependencies.repo.locations.column.snapshot                 = Snapshot
-report.dependencies.repo.locations.column.url                      = URL
-report.dependencies.repo.locations.title                           = Dependency Repository Locations
-report.dependencies.title                                          = Apache HBase&#8482; Dependencies
-report.dependencies.unnamed                                        = Unnamed
-report.dependencies.transitive.intro                               = This is a list of transitive dependencies for Apache HBase&#8482;. Transitive dependencies are the dependencies of the project dependencies.
-report.dependencies.transitive.nolist                              = No transitive dependencies are required for Apache HBase&#8482;.
-report.dependencies.transitive.title                               = Apache HBase&#8482; Transitive Dependencies
-report.dependency-convergence.convergence.caption                  = Dependencies used in modules
-report.dependency-convergence.convergence.single.caption           = Dependencies used in Apache HBase&#8482;
-report.dependency-convergence.description                          = This is the convergence of dependency versions across the entire project and its sub-modules.
-report.dependency-convergence.legend                               = Legend:
-report.dependency-convergence.legend.different                     = At least one dependency has a differing version of the dependency or has SNAPSHOT dependencies.
-report.dependency-convergence.legend.shared                        = All modules/dependencies share one version of the dependency.
-report.dependency-convergence.name                                 = Dependency Convergence
-report.dependency-convergence.reactor.name                         = Reactor Dependency Convergence
-report.dependency-convergence.reactor.title                        = Reactor Dependency Convergence
-report.dependency-convergence.stats.artifacts                      = Number of unique artifacts (NOA):
-report.dependency-convergence.stats.caption                        = Statistics:
-report.dependency-convergence.stats.convergence                    = Convergence (NOD/NOA):
-report.dependency-convergence.stats.dependencies                   = Number of dependencies (NOD):
-report.dependency-convergence.stats.readyrelease                   = Ready for release (100 % convergence and no SNAPSHOTS):
-report.dependency-convergence.stats.readyrelease.error             = Error
-report.dependency-convergence.stats.readyrelease.error.convergence = There is less than 100 % convergence.
-report.dependency-convergence.stats.readyrelease.error.snapshots   = There are SNAPSHOT dependencies.
-report.dependency-convergence.stats.readyrelease.success           = Success
-report.dependency-convergence.stats.conflicting                    = Number of version-conflicting artifacts (NOC):
-report.dependency-convergence.stats.snapshots                      = Number of SNAPSHOT artifacts (NOS):
-report.dependency-convergence.stats.modules                        = Number of modules:
-report.dependency-convergence.title                                = Dependency Convergence
-report.dependency-info.name                                        = Dependency Information
-report.dependency-info.title                                       = Dependency Information
-report.dependency-info.description                                 = These are instructions for including Apache HBase&#8482; as a dependency using various dependency management tools.
-report.index.nodescription                                         = There is currently no description associated with Apache HBase&#8482;.
-report.index.title                                                 = About Apache HBase&#8482;
-report.issuetracking.bugzilla.intro                                = Apache HBase&#8482; uses {Bugzilla, http://www.bugzilla.org/}.
-report.issuetracking.custom.intro                                  = Apache HBase&#8482; uses %issueManagementSystem% to manage its issues.
-report.issuetracking.description                                   = Apache HBase&#8482; uses the following issue management system(s).
-report.issuetracking.general.intro                                 = Apache HBase&#8482; uses an Issue Management System to manage its issues.
-report.issuetracking.intro                                         = Issues, bugs, and feature requests should be submitted to the following issue tracking system for Apache HBase&#8482;.
-report.issuetracking.jira.intro                                    = Apache HBase&#8482; uses {JIRA, http://www.atlassian.com/software/jira}.
-report.issuetracking.name                                          = Issue Tracking
-report.issuetracking.noissueManagement                             = No issue management system is defined. Please check back at a later date.
-report.issuetracking.overview.title                                = Overview
-report.issuetracking.scarab.intro                                  = Apache HBase&#8482; uses {Scarab, http://scarab.tigris.org/}.
-report.issuetracking.title                                         = Issue Tracking
-report.license.description                                         = Apache HBase&#8482; uses the following project license(s).
-report.license.multiple                                            = Apache HBase&#8482; is provided under multiple licenses:
-report.license.name                                                = Apache HBase&#8482; License
-report.license.nolicense                                           = No license is defined for Apache HBase&#8482;.
-report.license.overview.intro                                      = This is the license for the Apache HBase project itself, but not necessarily its dependencies.
-report.license.overview.title                                      = Overview
-report.license.originalText                                        = [Original text]
-report.license.copy                                                = Copy of the license follows:
-report.license.title                                               = Apache HBase&#8482; License
-report.license.unnamed                                             = Unnamed
-report.mailing-lists.column.archive                                = Archive
-report.mailing-lists.column.name                                   = Name
-report.mailing-lists.column.otherArchives                          = Other Archives
-report.mailing-lists.column.post                                   = Post
-report.mailing-lists.column.subscribe                              = Subscribe
-report.mailing-lists.column.unsubscribe                            = Unsubscribe
-report.mailing-lists.description                                   = These are Apache HBase&#8482;'s mailing lists.
-report.mailing-lists.intro                                         = For each list, links are provided to subscribe, unsubscribe, and view archives.
-report.mailing-lists.name                                          = Mailing Lists
-report.mailing-lists.nolist                                        = There are no mailing lists currently associated with Apache HBase&#8482;.
-report.mailing-lists.title                                         = Apache HBase&#8482; Mailing Lists
-report.scm.accessbehindfirewall.cvs.intro                          = If you are behind a firewall that blocks HTTP access to the CVS repository, you can use the {CVSGrab, http://cvsgrab.sourceforge.net/} web interface to checkout the source code.
-report.scm.accessbehindfirewall.general.intro                      = Refer to the documentation of the SCM used for more information about access behind a firewall.
-report.scm.accessbehindfirewall.svn.intro                          = If you are behind a firewall that blocks HTTP access to the Subversion repository, you can try to access it via the developer connection:
-report.scm.accessbehindfirewall.title                              = Access from Behind a Firewall
-report.scm.accessthroughtproxy.svn.intro1                          = The Subversion client can go through a proxy, if you configure it to do so. First, edit your "servers" configuration file to indicate which proxy to use. The file's location depends on your operating system. On Linux or Unix it is located in the directory "~/.subversion". On Windows it is in "%APPDATA%\\Subversion". (Try "echo %APPDATA%", note this is a hidden directory.)
-report.scm.accessthroughtproxy.svn.intro2                          = There are comments in the file explaining what to do. If you don't have that file, get the latest Subversion client and run any command; this will cause the configuration directory and template files to be created.
-report.scm.accessthroughtproxy.svn.intro3                          = Example: Edit the 'servers' file and add something like:
-report.scm.accessthroughtproxy.title                               = Access Through a Proxy
-report.scm.anonymousaccess.cvs.intro                               = Apache HBase&#8482;'s CVS repository can be checked out through anonymous CVS with the following instruction set. When prompted for a password for anonymous, simply press the Enter key.
-report.scm.anonymousaccess.general.intro                           = Refer to the documentation of the SCM used for more information about anonymously check out. The connection url is:
-report.scm.anonymousaccess.git.intro                               = The source can be checked out anonymously from Git with this command (See {http://git-scm.com/docs/git-clone,http://git-scm.com/docs/git-clone}):
-report.scm.anonymousaccess.hg.intro                                = The source can be checked out anonymously from Mercurial with this command (See {http://www.selenic.com/mercurial/hg.1.html#clone,http://www.selenic.com/mercurial/hg.1.html#clone}):
-report.scm.anonymousaccess.svn.intro                               = The source can be checked out anonymously from Subversion with this command:
-report.scm.anonymousaccess.title                                   = Anonymous Access
-report.scm.clearcase.intro                                         = Apache HBase&#8482; uses {ClearCase, http://www-306.ibm.com/software/awdtools/clearcase/} to manage its source code. Informations on ClearCase use can be found at {http://www.redbooks.ibm.com/redbooks/pdfs/sg246399.pdf, http://www.redbooks.ibm.com/redbooks/pdfs/sg246399.pdf}.
-report.scm.cvs.intro                                               = Apache HBase&#8482; uses {Concurrent Versions System, http://www.cvshome.org/} to manage its source code. Instructions on CVS use can be found at {http://cvsbook.red-bean.com/, http://cvsbook.red-bean.com/}.
-report.scm.description                                             = This document lists ways to access the online source repository.
-report.scm.devaccess.clearcase.intro                               = Only project developers can access the ClearCase tree via this method. Substitute username with the proper value.
-report.scm.devaccess.cvs.intro                                     = Only project developers can access the CVS tree via this method. Substitute username with the proper value.
-report.scm.devaccess.general.intro                                 = Refer to the documentation of the SCM used for more information about developer check out. The connection url is:
-report.scm.devaccess.git.intro                                     = Only project developers can access the Git tree via this method (See {http://git-scm.com/docs/git-clone,http://git-scm.com/docs/git-clone}).
-report.scm.devaccess.hg.intro                                      = Only project developers can access the Mercurial tree via this method (See {http://www.selenic.com/mercurial/hg.1.html#clone,http://www.selenic.com/mercurial/hg.1.html#clone}).
-report.scm.devaccess.perforce.intro                                = Only project developers can access the Perforce tree via this method. Substitute username and password with the proper values.
-report.scm.devaccess.starteam.intro                                = Only project developers can access the Starteam tree via this method. Substitute username with the proper value.
-report.scm.devaccess.svn.intro1.https                              = Everyone can access the Subversion repository via HTTP, but committers must checkout the Subversion repository via HTTPS.
-report.scm.devaccess.svn.intro1.other                              = Committers must checkout the Subversion repository.
-report.scm.devaccess.svn.intro1.svn                                = Committers must checkout the Subversion repository via SVN.
-report.scm.devaccess.svn.intro1.svnssh                             = Committers must checkout the Subversion repository via SVN+SSH.
-report.scm.devaccess.svn.intro2                                    = To commit changes to the repository, execute the following command to commit your changes (svn will prompt you for your password):
-report.scm.devaccess.title                                         = Developer Access
-report.scm.general.intro                                           = Apache HBase&#8482; uses a Source Content Management System to manage its source code.
-report.scm.name                                                    = Source Repository
-report.scm.noscm                                                   = No source configuration management system is defined. Please check back at a later date.
-report.scm.overview.title                                          = Overview
-report.scm.git.intro                                               = Apache HBase&#8482; uses {Git, http://git-scm.com/} to manage its source code. Instructions on Git use can be found at {http://git-scm.com/documentation,http://git-scm.com/documentation}.
-report.scm.hg.intro                                                = Apache HBase&#8482; uses {Mercurial, http://mercurial.selenic.com/wiki/} to manage its source code. Instructions on Mercurial use can be found at {http://hgbook.red-bean.com/read/, http://hgbook.red-bean.com/read/}.
-report.scm.perforce.intro                                          = Apache HBase&#8482; uses {Perforce, http://www.perforce.com/} to manage its source code. Instructions on Perforce use can be found at {http://www.perforce.com/perforce/doc.051/manuals/cmdref/index.html, http://www.perforce.com/perforce/doc.051/manuals/cmdref/index.html}.
-report.scm.starteam.intro                                          = Apache HBase&#8482; uses {Starteam, http://www.borland.com/us/products/starteam/} to manage its source code.
-report.scm.svn.intro                                               = Apache HBase&#8482; uses {Subversion, http://subversion.apache.org/} to manage its source code. Instructions on Subversion use can be found at {http://svnbook.red-bean.com/, http://svnbook.red-bean.com/}.
-report.scm.title                                                   = Source Repository
-report.scm.webaccess.nourl                                         = There is no browsable version of the source repository listed for Apache HBase&#8482;. Please check back again later.
-report.scm.webaccess.title                                         = Web Browser Access
-report.scm.webaccess.url                                           = The following is a link to a browsable version of the source repository:
-report.summary.build.artifactid                                    = ArtifactId
-report.summary.build.groupid                                       = GroupId
-report.summary.build.javaVersion                                   = Java Version
-report.summary.build.title                                         = Build Information
-report.summary.build.type                                          = Type
-report.summary.build.version                                       = Version
-report.summary.description                                         = This document lists other related information of Apache HBase&#8482;
-report.summary.field                                               = Field
-report.summary.general.description                                 = Description
-report.summary.general.homepage                                    = Homepage
-report.summary.general.name                                        = Name
-report.summary.general.title                                       = Project Information
-report.summary.name                                                = Project Summary
-report.summary.organization.name                                   = Name
-report.summary.organization.title                                  = Project Organization
-report.summary.organization.url                                    = URL
-report.summary.noorganization                                      = Apache HBase&#8482; does not belong to an organization.
-report.summary.title                                               = Project Summary
-report.summary.value                                               = Value
-report.summary.download                                            = Download
-report.team-list.contributors.actualtime                           = Actual Time (GMT)
-report.team-list.contributors.email                                = Email
-report.team-list.contributors.intro                                = The following additional people have contributed to Apache HBase&#8482; through the way of suggestions, patches or documentation.
-report.team-list.contributors.image                                = Image
-report.team-list.contributors.name                                 = Name
-report.team-list.contributors.organization                         = Organization
-report.team-list.contributors.organizationurl                      = Organization URL
-report.team-list.contributors.properties                           = Properties
-report.team-list.contributors.roles                                = Roles
-report.team-list.contributors.timezone                             = Time Zone
-report.team-list.contributors.title                                = Contributors
-report.team-list.contributors.url                                  = URL
-report.team-list.description                                       = These are the members of the Apache HBase&#8482; project. These are the individuals who have contributed to the project in one form or another.
-report.team-list.developers.actualtime                             = Actual Time (GMT)
-report.team-list.developers.email                                  = Email
-report.team-list.developers.image                                  = Image
-report.team-list.developers.id                                     = Id
-report.team-list.developers.intro                                  = These are the developers with commit privileges that have directly contributed to the project in one way or another.
-report.team-list.developers.name                                   = Name
-report.team-list.developers.organization                           = Organization
-report.team-list.developers.organizationurl                        = Organization URL
-report.team-list.developers.properties                             = Properties
-report.team-list.developers.roles                                  = Roles
-report.team-list.developers.timezone                               = Time Zone
-report.team-list.developers.title                                  = Members
-report.team-list.developers.url                                    = URL
-report.team-list.intro.description1                                = A successful project requires many people to play many roles. Some members write code or documentation, while others are valuable as testers, submitting patches and suggestions.
-report.team-list.intro.description2                                = The team is comprised of Members and Contributors. Members have direct access to the source of a project and actively evolve the code-base. Contributors improve the project through submission of patches and suggestions to the Members. The number of Contributors to the project is unbounded. Get involved today. All contributions to the project are greatly appreciated.
-report.team-list.intro.title                                       = The Team
-report.team-list.name                                              = Project Team
-report.team-list.nocontributor                                     = Apache HBase&#8482; does not maintain a list of contributors.
-report.team-list.nodeveloper                                       = Apache HBase&#8482; does not maintain a list of developers.
-report.team-list.title                                             = Project Team
-report.dependencyManagement.name                                   = Dependency Management
-report.dependencyManagement.description                            = This document lists the dependencies that are defined through dependencyManagement.
-report.dependencyManagement.title                                  = Project Dependency Management
-report.dependencyManagement.nolist                                 = There are no dependencies in the DependencyManagement of Apache HBase&#8482;.
-report.dependencyManagement.column.groupId                         = GroupId
-report.dependencyManagement.column.artifactId                      = ArtifactId
-report.dependencyManagement.column.version                         = Version
-report.dependencyManagement.column.classifier                      = Classifier
-report.dependencyManagement.column.type                            = Type
-report.dependencyManagement.column.license                         = License
-report.dependencyManagement.intro.compile                          = The following is a list of compile dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile and run the submodule:
-report.dependencyManagement.intro.provided                         = The following is a list of provided dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile the submodule, but should be provided by default when using the library:
-report.dependencyManagement.intro.runtime                          = The following is a list of runtime dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to run the submodule:
-report.dependencyManagement.intro.system                           = The following is a list of system dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile the submodule:
-report.dependencyManagement.intro.test                             = The following is a list of test dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile and run unit tests for the submodule:
-report.pluginManagement.nolist                                     = There are no plugins defined in the PluginManagement part of Apache HBase&#8482;.
-report.pluginManagement.name                                       = Plugin Management
-report.pluginManagement.description                                = This document lists the plugins that are defined through pluginManagement.
-report.pluginManagement.title                                      = Project Plugin Management
-report.plugins.name                                                = Project Plugins
-report.plugins.description                                         = This document lists the build plugins and the report plugins used by Apache HBase&#8482;.
-report.plugins.title                                               = Project Build Plugins
-report.plugins.report.title                                        = Project Report Plugins
-report.plugins.nolist                                              = There are no plugins defined in the Build part of Apache HBase&#8482;.
-report.plugins.report.nolist                                       = There are no plugins reports defined in the Reporting part of Apache HBase&#8482;.
-report.modules.nolist                                              = There are no modules declared in Apache HBase&#8482;.
-report.modules.name                                                = Project Modules
-report.modules.description                                         = This document lists the modules (sub-projects) of Apache HBase&#8482;.
-report.modules.title                                               = Project Modules
-report.modules.intro                                               = Apache HBase&#8482; has declared the following modules:
-report.modules.header.name                                         = Name
-report.modules.header.description                                  = Description
-report.distributionManagement.name                                 = Distribution Management
-report.distributionManagement.description                          = This document provides informations on the distribution management of Apache HBase&#8482;.
-report.distributionManagement.title                                = Project Distribution Management
-report.distributionManagement.nodistributionmanagement             = No distribution management is defined for Apache HBase&#8482;.
-report.distributionManagement.overview.title                       = Overview
-report.distributionManagement.overview.intro                       = The following is the distribution management information used by Apache HBase&#8482;.
-report.distributionManagement.downloadURL                          = Download URL
-report.distributionManagement.repository                           = Repository
-report.distributionManagement.snapshotRepository                   = Snapshot Repository
-report.distributionManagement.site                                 = Site
-report.distributionManagement.relocation                           = Relocation
-report.distributionManagement.field                                = Field
-report.distributionManagement.value                                = Value
-report.distributionManagement.relocation.groupid                   = GroupId
-report.distributionManagement.relocation.artifactid                = ArtifactId
-report.distributionManagement.relocation.version                   = Version
-report.distributionManagement.relocation.message                   = Message

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/.htaccess
----------------------------------------------------------------------
diff --git a/src/main/site/resources/.htaccess b/src/main/site/resources/.htaccess
deleted file mode 100644
index 20bf651..0000000
--- a/src/main/site/resources/.htaccess
+++ /dev/null
@@ -1,8 +0,0 @@
-
-# Redirect replication URL to the right section of the book
-# Rule added 2015-1-12 -- can be removed in 6 months
-Redirect permanent /replication.html /book.html#_cluster_replication
-
-# Redirect old page-per-chapter book sections to new single file.
-RedirectMatch permanent ^/book/(.*)\.html$ /book.html#$1
-RedirectMatch permanent ^/book/$ /book.html

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/book/.empty
----------------------------------------------------------------------
diff --git a/src/main/site/resources/book/.empty b/src/main/site/resources/book/.empty
deleted file mode 100644
index 5513814..0000000
--- a/src/main/site/resources/book/.empty
+++ /dev/null
@@ -1 +0,0 @@
-# This directory is here so that we can have rewrite rules in our .htaccess to maintain old links. Otherwise we fall under some top-level niceness redirects because we have a file named book.html.

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/src/main/site/resources/css/site.css b/src/main/site/resources/css/site.css
deleted file mode 100644
index 3f42f5a..0000000
--- a/src/main/site/resources/css/site.css
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.
- */
-
-/*@import(https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap.min.css);
-@import(https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap-responsive.css);*/
-html {
-  background-color: #fff;
-}
-body {
-  font-size: 16px;
-}
-li {
-  line-height: 120%;
-}
-
-div#topbar,
-div#banner,
-div#breadcrumbs,
-div#bodyColumn,
-footer {
-  width: initial;
-  padding-left: 20px;
-  padding-right: 20px;
-  clear: both;
-}
-footer {
-  background-color: #e5e5e5;
-}
-footer .row, footer p, footer .pull-right {
-  margin: 5px;
-}
-div#search-form.navbar-search.pull-right {
-  width: 290px;
-  margin-right: 0;
-  margin-top: -5px;
-  margin-left: 0;
-  position: initial;
-}
-li#publishDate.pull-right {
-  list-style: none;
-}
-.container,
-.navbar-static-top .container,
-.navbar-fixed-top .container,
-.navbar-fixed-bottom .container,
-.navbar-inner {
-  width: initial;
-}
-/* Change the color and effect when clicking in menus */
-.dropdown-menu>li>a:hover,
-.dropdown-menu>li>a:focus,
-.dropdown-submenu:hover>a,
-.dropdown-submenu:focus>a {
-  background-color: #e5e5e5;
-  background-image: none;
-  color: #000;
-  font-weight: bolder;
-}
-
-.dropdown-backdrop {
-  position: static;
-}
-
-@media only screen and (max-width: 979px) {
-  body {
-    padding-left: 0;
-    padding-right: 0;
-    width: initial;
-    margin: 0;
-  }
-  /* Without this rule, drop-down divs are a fixed height
-   * the first time they are expanded */
-  .collapse.in {
-      height: auto !important;
-  }
-  div#search-form.navbar-search.pull-right {
-    padding: 0;
-    margin-left: ;
-    width: initial;
-    clear: both;
-  }
-}
-
-/* Fix Google Custom Search results on very narrow screens */
-@media(max-width: 480px) {
-    .gsc-overflow-hidden .nav-collapse {
-        -webkit-transform: none;
-    }
-}
-
-/* Override weird body padding thing that causes scrolling */
-@media (max-width: 767px)
-body {
-    padding-right: 0;
-    padding-left: 0;
-}
-
-@media (max-width: 767px)
-.navbar-fixed-top, .navbar-fixed-bottom, .navbar-static-top {
-  margin-left: 0;
-  margin-right: 0;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/doap_Hbase.rdf
----------------------------------------------------------------------
diff --git a/src/main/site/resources/doap_Hbase.rdf b/src/main/site/resources/doap_Hbase.rdf
deleted file mode 100644
index 46082a1..0000000
--- a/src/main/site/resources/doap_Hbase.rdf
+++ /dev/null
@@ -1,57 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl"?>
-<rdf:RDF xml:lang="en"
-         xmlns="http://usefulinc.com/ns/doap#" 
-         xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#" 
-         xmlns:asfext="http://projects.apache.org/ns/asfext#"
-         xmlns:foaf="http://xmlns.com/foaf/0.1/">
-<!--
-    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.
--->
-  <Project rdf:about="http://hbase.apache.org">
-    <created>2012-04-14</created>
-    <license rdf:resource="http://usefulinc.com/doap/licenses/asl20" />
-    <name>Apache HBase</name>
-    <homepage rdf:resource="http://hbase.apache.org" />
-    <asfext:pmc rdf:resource="http://hbase.apache.org" />
-    <shortdesc>Apache HBase software is the Hadoop database. Think of it as a distributed, scalable, big data store.</shortdesc>
-    <description>Use Apache HBase software when you need random, realtime read/write access to your Big Data. This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware. HBase is an open-source, distributed, versioned, column-oriented store modeled after Google's Bigtable: A Distributed Storage System for Structured Data by Chang et al. Just as Bigtable leverages the distributed data storage provided by the Google File System, HBase provides Bigtable-like capabilities on top of Hadoop and HDFS. </description>
-    <bug-database rdf:resource="http://issues.apache.org/jira/browse/HBASE" />
-    <mailing-list rdf:resource="http://hbase.apache.org/mail-lists.html" />
-    <download-page rdf:resource="http://www.apache.org/dyn/closer.cgi/hbase/" />
-    <programming-language>Java</programming-language>
-    <category rdf:resource="http://projects.apache.org/category/database" />
-    <release>
-      <Version>
-        <name>Apache hbase </name>
-        <created>2015-07-23</created>
-        <revision>2.0.0-SNAPSHOT</revision>
-      </Version>
-    </release>
-    <repository>
-      <GitRepository>
-        <location rdf:resource="git://git.apache.org/hbase.git"/>
-        <browse rdf:resource="https://git-wip-us.apache.org/repos/asf?p=hbase.git"/>
-      </GitRepository>
-    </repository>
-    <maintainer>
-      <foaf:Person>
-        <foaf:name>Apache HBase PMC</foaf:name>
-          <foaf:mbox rdf:resource="mailto:dev@hbase.apache.org"/>
-      </foaf:Person>
-    </maintainer>
-  </Project>
-</rdf:RDF>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/architecture.gif
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/architecture.gif b/src/main/site/resources/images/architecture.gif
deleted file mode 100644
index 8d84a23..0000000
Binary files a/src/main/site/resources/images/architecture.gif and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/bc_basic.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/bc_basic.png b/src/main/site/resources/images/bc_basic.png
deleted file mode 100644
index 231de93..0000000
Binary files a/src/main/site/resources/images/bc_basic.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/bc_config.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/bc_config.png b/src/main/site/resources/images/bc_config.png
deleted file mode 100644
index 53250cf..0000000
Binary files a/src/main/site/resources/images/bc_config.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/bc_l1.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/bc_l1.png b/src/main/site/resources/images/bc_l1.png
deleted file mode 100644
index 36d7e55..0000000
Binary files a/src/main/site/resources/images/bc_l1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/bc_l2_buckets.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/bc_l2_buckets.png b/src/main/site/resources/images/bc_l2_buckets.png
deleted file mode 100644
index 5163928..0000000
Binary files a/src/main/site/resources/images/bc_l2_buckets.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/bc_stats.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/bc_stats.png b/src/main/site/resources/images/bc_stats.png
deleted file mode 100644
index d8c6384..0000000
Binary files a/src/main/site/resources/images/bc_stats.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/big_h_logo.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/big_h_logo.png b/src/main/site/resources/images/big_h_logo.png
deleted file mode 100644
index 5256094..0000000
Binary files a/src/main/site/resources/images/big_h_logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/big_h_logo.svg
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/big_h_logo.svg b/src/main/site/resources/images/big_h_logo.svg
deleted file mode 100644
index ab24198..0000000
--- a/src/main/site/resources/images/big_h_logo.svg
+++ /dev/null
@@ -1,139 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!-- Generator: Adobe Illustrator 15.1.0, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
-
-<svg
-   xmlns:dc="http://purl.org/dc/elements/1.1/"
-   xmlns:cc="http://creativecommons.org/ns#"
-   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
-   xmlns:svg="http://www.w3.org/2000/svg"
-   xmlns="http://www.w3.org/2000/svg"
-   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
-   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
-   version="1.1"
-   id="Layer_1"
-   x="0px"
-   y="0px"
-   width="792px"
-   height="612px"
-   viewBox="0 0 792 612"
-   enable-background="new 0 0 792 612"
-   xml:space="preserve"
-   inkscape:version="0.48.4 r9939"
-   sodipodi:docname="big_h_same_font_hbase3_logo.png"
-   inkscape:export-filename="big_h_bitmap.png"
-   inkscape:export-xdpi="90"
-   inkscape:export-ydpi="90"><metadata
-   id="metadata3693"><rdf:RDF><cc:Work
-       rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
-         rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
-   id="defs3691" /><sodipodi:namedview
-   pagecolor="#000000"
-   bordercolor="#666666"
-   borderopacity="1"
-   objecttolerance="10"
-   gridtolerance="10"
-   guidetolerance="10"
-   inkscape:pageopacity="0"
-   inkscape:pageshadow="2"
-   inkscape:window-width="1440"
-   inkscape:window-height="856"
-   id="namedview3689"
-   showgrid="false"
-   inkscape:zoom="2.1814013"
-   inkscape:cx="415.39305"
-   inkscape:cy="415.72702"
-   inkscape:window-x="1164"
-   inkscape:window-y="22"
-   inkscape:window-maximized="0"
-   inkscape:current-layer="Layer_1" />
-
-
-
-
-
-
-<text
-   xml:space="preserve"
-   style="font-size:40px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
-   x="311.18643"
-   y="86.224579"
-   id="text3082"
-   sodipodi:linespacing="125%"><tspan
-     sodipodi:role="line"
-     id="tspan3084"
-     x="311.18643"
-     y="86.224579" /></text>
-<text
-   xml:space="preserve"
-   style="font-size:40px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
-   x="283.95764"
-   y="87.845337"
-   id="text3086"
-   sodipodi:linespacing="125%"><tspan
-     sodipodi:role="line"
-     id="tspan3088"
-     x="283.95764"
-     y="87.845337" /></text>
-<g
-   id="g3105"
-   transform="translate(14.669469,-80.682082)"
-   inkscape:export-filename="/Users/stack/Documents/big_h_base.png"
-   inkscape:export-xdpi="90"
-   inkscape:export-ydpi="90"><path
-     sodipodi:nodetypes="ccccccccccccccccccccccccccccc"
-     style="fill:#ba160c"
-     inkscape:connector-curvature="0"
-     id="path3677"
-     d="m 589.08202,499.77746 -40.3716,0 0,-168.36691 40.3716,0 z m -40.20304,-168.35619 -0.1684,-104.30857 40.3716,0 -0.33048,104.26805 z m -0.1684,168.35619 -40.37568,0 0,-104.82988 -259.42272,0 0,104.82988 -79.42128,0 0,-272.66476 79.42128,0 0,104.29785 224.92224,0 34.50456,0 40.37568,0 0,168.36691 z m 0,-272.66476 -40.37568,0 -0.0171,104.30857 40.55802,-0.01 z"
-     inkscape:export-filename="/Users/stack/Documents/polygon3687.png"
-     inkscape:export-xdpi="90"
-     inkscape:export-ydpi="90" /><path
-     sodipodi:nodetypes="cscsccsssccsssccscsccccccccccccccccccccc"
-     style="fill:#ba160c"
-     inkscape:connector-curvature="0"
-     id="path3679"
-     d="m 263.96692,553.27262 c 6.812,4.218 10.219,10.652 10.219,19.303 0,6.272 -2,11.571 -6.002,15.897 -4.325,4.758 -10.165,7.137 -17.519,7.137 l -28.629,0 0,-19.465 28.629,0 c 2.812,0 4.218,-2.109 4.218,-6.327 0,-4.216 -1.406,-6.325 -4.218,-6.325 l -28.629,0 0,-19.303 27.17,0 c 2.811,0 4.217,-2.109 4.217,-6.327 0,-4.216 -1.406,-6.326 -4.217,-6.326 l -27.17,0 0,-19.464 27.17,0 c 7.353,0 13.192,2.379 17.519,7.137 3.892,4.325 5.839,9.625 5.839,15.896 0,7.787 -2.866,13.842 -8.597,18.167 z m -41.931,42.338 -52.312,0 0,-51.42 19.466,0 5.259,0 27.588,0 0,19.303 -32.847,0 0,12.652 32.847,0 0,19.465 z m 0,-64.073 -32.847,0 0.0405,12.76351 -19.466,0.081 -0.0405,-32.30954 52.312,0 0,19.465 z" /><path
-     style="fill:#ba160c"
-     inkscape:connector-curvature="0"
-     id="path3683"
-     d="m 384.35292,595.61062 h -19.465 v -26.602 h -31.094 -0.618 v -19.466 h 0.618 31.094 v -11.68 c 0,-4.216 -1.406,-6.324 -4.218,-6.324 h -27.494 v -19.465 h 27.494 c 7.03,0 12.733,2.541 17.114,7.623 4.379,5.083 6.569,11.139 6.569,18.167 v 57.747 z m -51.177,-26.602 h -19.547 -12.165 v 26.602 h -19.466 v -57.748 c 0,-7.028 2.19,-13.083 6.569,-18.167 4.379,-5.083 10.03,-7.623 16.952,-7.623 h 27.656 v 19.466 h -27.656 c -2.704,0 -4.055,2.108 -4.055,6.324 v 11.68 h 12.165 19.547 v 19.466 z" /><path
-     style="fill:#ba160c"
-     inkscape:connector-curvature="0"
-     id="path3685"
-     d="m 492.35692,569.81862 c 0,7.03 -2.109,13.031 -6.327,18.006 -4.541,5.19 -10.273,7.786 -17.193,7.786 h -72.02 v -19.465 h 72.02 c 2.704,0 4.055,-2.109 4.055,-6.327 0,-4.216 -1.352,-6.325 -4.055,-6.325 h -52.394 c -6.92,0 -12.652,-2.596 -17.193,-7.787 -4.327,-4.865 -6.49,-10.813 -6.49,-17.843 0,-7.028 2.218,-13.083 6.651,-18.167 4.434,-5.083 10.112,-7.623 17.032,-7.623 h 72.021 v 19.464 h -72.021 c -2.703,0 -4.055,2.109 -4.055,6.326 0,4.109 1.352,6.164 4.055,6.164 h 52.394 c 6.92,0 12.652,2.596 17.193,7.787 4.218,4.974 6.327,10.976 6.327,18.004 z" /><polygon
-     style="fill:#ba160c"
-     transform="translate(-71.972085,223.93862)"
-     id="polygon3687"
-     points="656.952,339.555 591.906,339.555 591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 "
-     inkscape:export-xdpi="90"
-     inkscape:export-ydpi="90" /><g
-     id="g3349"><g
-       id="g3344"><text
-         transform="scale(0.93350678,1.0712295)"
-         sodipodi:linespacing="125%"
-         id="text3076"
-         y="203.03328"
-         x="181.98402"
-         style="font-size:84.015625px;font-style:italic;font-variant:normal;font-weight:bold;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#4d4d4d;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi Bold Italic"
-         xml:space="preserve"
-         inkscape:export-xdpi="90"
-         inkscape:export-ydpi="90"
-         inkscape:export-filename="/Users/stack/Documents/polygon3687.png"><tspan
-           style="font-size:84.015625px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:25.64349174px;writing-mode:lr-tb;text-anchor:start;fill:#4d4d4d;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
-           y="203.03328"
-           x="181.98402"
-           id="tspan3080"
-           sodipodi:role="line">APACHE</tspan></text>
-<rect
-         y="191.93103"
-         x="178.85117"
-         height="10.797735"
-         width="7.7796612"
-         id="rect3090"
-         style="fill:#4d4d4d" /></g><rect
-       style="fill:#4d4d4d"
-       id="rect3103"
-       width="8.1443329"
-       height="10.787481"
-       x="334.64697"
-       y="191.93881" /></g></g></svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/data_block_diff_encoding.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/data_block_diff_encoding.png b/src/main/site/resources/images/data_block_diff_encoding.png
deleted file mode 100644
index 0bd03a4..0000000
Binary files a/src/main/site/resources/images/data_block_diff_encoding.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/data_block_no_encoding.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/data_block_no_encoding.png b/src/main/site/resources/images/data_block_no_encoding.png
deleted file mode 100644
index 56498b4..0000000
Binary files a/src/main/site/resources/images/data_block_no_encoding.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/data_block_prefix_encoding.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/data_block_prefix_encoding.png b/src/main/site/resources/images/data_block_prefix_encoding.png
deleted file mode 100644
index 4271847..0000000
Binary files a/src/main/site/resources/images/data_block_prefix_encoding.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/favicon.ico
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/favicon.ico b/src/main/site/resources/images/favicon.ico
deleted file mode 100644
index 6e4d0f7..0000000
Binary files a/src/main/site/resources/images/favicon.ico and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hadoop-logo.jpg
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hadoop-logo.jpg b/src/main/site/resources/images/hadoop-logo.jpg
deleted file mode 100644
index 809525d..0000000
Binary files a/src/main/site/resources/images/hadoop-logo.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbase_logo.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_logo.png b/src/main/site/resources/images/hbase_logo.png
deleted file mode 100644
index e962ce0..0000000
Binary files a/src/main/site/resources/images/hbase_logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbase_logo.svg
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_logo.svg b/src/main/site/resources/images/hbase_logo.svg
deleted file mode 100644
index 2cc26d9..0000000
--- a/src/main/site/resources/images/hbase_logo.svg
+++ /dev/null
@@ -1,78 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="no"?>
-<!-- Generator: Adobe Illustrator 15.1.0, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
-
-<svg
-   xmlns:dc="http://purl.org/dc/elements/1.1/"
-   xmlns:cc="http://creativecommons.org/ns#"
-   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
-   xmlns:svg="http://www.w3.org/2000/svg"
-   xmlns="http://www.w3.org/2000/svg"
-   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
-   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
-   version="1.1"
-   id="Layer_1"
-   x="0px"
-   y="0px"
-   width="792px"
-   height="612px"
-   viewBox="0 0 792 612"
-   enable-background="new 0 0 792 612"
-   xml:space="preserve"
-   inkscape:version="0.48.4 r9939"
-   sodipodi:docname="hbase_banner_logo.png"
-   inkscape:export-filename="hbase_logo_filledin.png"
-   inkscape:export-xdpi="90"
-   inkscape:export-ydpi="90"><metadata
-   id="metadata3285"><rdf:RDF><cc:Work
-       rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
-         rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
-   id="defs3283" /><sodipodi:namedview
-   pagecolor="#ffffff"
-   bordercolor="#666666"
-   borderopacity="1"
-   objecttolerance="10"
-   gridtolerance="10"
-   guidetolerance="10"
-   inkscape:pageopacity="0"
-   inkscape:pageshadow="2"
-   inkscape:window-width="1131"
-   inkscape:window-height="715"
-   id="namedview3281"
-   showgrid="false"
-   inkscape:zoom="4.3628026"
-   inkscape:cx="328.98554"
-   inkscape:cy="299.51695"
-   inkscape:window-x="752"
-   inkscape:window-y="456"
-   inkscape:window-maximized="0"
-   inkscape:current-layer="Layer_1" />
-<path
-   d="m 233.586,371.672 -9.895,0 0,-51.583 9.895,0 0,51.583 z m -9.77344,-51.59213 -0.12156,-31.94487 9.895,0 -0.0405,31.98539 z m -0.12156,51.59213 -9.896,0 0,-32.117 -63.584,0 0,32.117 -19.466,0 0,-83.537 19.466,0 0,31.954 55.128,0 8.457,0 9.896,0 0,51.583 z m 0,-83.537 -9.896,0 0,31.98539 10.01756,-0.0405 z"
-   id="path3269"
-   inkscape:connector-curvature="0"
-   style="fill:#ba160c"
-   sodipodi:nodetypes="cccccccccccccccccccccccccccccc" />
-<path
-   d="m 335.939,329.334 c 6.812,4.218 10.219,10.652 10.219,19.303 0,6.272 -2,11.571 -6.002,15.897 -4.325,4.758 -10.165,7.137 -17.519,7.137 l -28.629,0 0,-19.465 28.629,0 c 2.812,0 4.218,-2.109 4.218,-6.327 0,-4.216 -1.406,-6.325 -4.218,-6.325 l -28.629,0 0,-19.303 27.17,0 c 2.811,0 4.217,-2.109 4.217,-6.327 0,-4.216 -1.406,-6.326 -4.217,-6.326 l -27.17,0 0,-19.464 27.17,0 c 7.353,0 13.192,2.379 17.519,7.137 3.892,4.325 5.839,9.625 5.839,15.896 0,7.787 -2.866,13.842 -8.597,18.167 z m -41.931,42.338 -52.312,0 0,-51.42 19.466,0 5.259,0 27.588,0 0,19.303 -32.847,0 0,12.652 32.847,0 0,19.465 z m 0,-64.073 -32.847,0 0.0405,13.24974 -19.466,-0.48623 -0.0405,-32.22851 52.312,0 0,19.465 z"
-   id="path3271"
-   inkscape:connector-curvature="0"
-   style="fill:#ba160c"
-   sodipodi:nodetypes="cscsccsssccsssccscsccccccccccccccccccccc" />
-<path
-   d="M355.123,266.419v-8.92h14.532v-5.353c0-1.932-0.644-2.899-1.933-2.899h-12.6v-8.919h12.6  c3.223,0,5.836,1.164,7.842,3.494c2.007,2.33,3.011,5.104,3.011,8.325v26.463h-8.921v-12.19H355.123L355.123,266.419z   M473.726,278.61h-29.587c-3.469,0-6.417-1.152-8.845-3.458c-2.429-2.304-3.642-5.191-3.642-8.659v-14.049  c0-3.47,1.213-6.356,3.642-8.662c2.428-2.304,5.376-3.455,8.845-3.455h29.587v8.919h-29.587c-2.378,0-3.567,1.066-3.567,3.197  v14.049c0,2.131,1.189,3.196,3.567,3.196h29.587V278.61L473.726,278.61z M567.609,278.61h-8.996v-14.718h-22.895v14.718h-8.92  v-38.282h8.92v14.644h22.895v-14.644h8.996V278.61L567.609,278.61z M661.494,249.247h-31.889v5.725h29.807v8.92h-29.807v5.797  h31.814v8.92h-40.735v-38.282h40.809V249.247z M355.123,240.328v8.919h-12.674c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h2.435  h6.522v8.92h-6.522h-2.435h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.011-8.325c2.006-2.33,4.596-3.494,7.768-3.494H355.123  L355.123,240.328z M254.661,266.122v-8.92h13.083c1.288,0,1.
 933-1.313,1.933-3.939c0-2.676-0.645-4.015-1.933-4.015h-13.083v-8.919  h13.083c3.32,0,5.995,1.363,8.028,4.088c1.883,2.478,2.825,5.425,2.825,8.846c0,3.419-0.942,6.342-2.825,8.771  c-2.033,2.725-4.708,4.088-8.028,4.088H254.661z M177.649,278.61h-8.92v-12.19h-14.532v-8.92h14.532v-5.353  c0-1.932-0.644-2.899-1.932-2.899h-12.6v-8.919h12.6c3.222,0,5.835,1.164,7.842,3.494c2.007,2.33,3.01,5.104,3.01,8.325V278.61  L177.649,278.61z M254.661,240.328v8.919h-15.016v7.954h15.016v8.92h-15.016v12.488h-8.92v-38.282H254.661z M154.198,266.419h-7.604  h-1.354h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.01-8.325c2.007-2.33,4.597-3.494,7.768-3.494h12.674v8.919h-12.674  c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h1.354h7.604V266.419z"
-   id="path3273"
-   style="fill:#666666"
-   fill="#878888" />
-<path
-   fill="#BA160C"
-   d="M456.325,371.672H436.86V345.07h-31.094h-0.618v-19.466h0.618h31.094v-11.68  c0-4.216-1.406-6.324-4.218-6.324h-27.494v-19.465h27.494c7.03,0,12.733,2.541,17.114,7.623c4.379,5.083,6.569,11.139,6.569,18.167  V371.672z M405.148,345.07h-19.547h-12.165v26.602h-19.466v-57.748c0-7.028,2.19-13.083,6.569-18.167  c4.379-5.083,10.03-7.623,16.952-7.623h27.656V307.6h-27.656c-2.704,0-4.055,2.108-4.055,6.324v11.68h12.165h19.547V345.07z"
-   id="path3275" />
-<path
-   fill="#BA160C"
-   d="M564.329,345.88c0,7.03-2.109,13.031-6.327,18.006c-4.541,5.19-10.273,7.786-17.193,7.786h-72.02v-19.465  h72.02c2.704,0,4.055-2.109,4.055-6.327c0-4.216-1.352-6.325-4.055-6.325h-52.394c-6.92,0-12.652-2.596-17.193-7.787  c-4.327-4.865-6.49-10.813-6.49-17.843c0-7.028,2.218-13.083,6.651-18.167c4.434-5.083,10.112-7.623,17.032-7.623h72.021v19.464  h-72.021c-2.703,0-4.055,2.109-4.055,6.326c0,4.109,1.352,6.164,4.055,6.164h52.394c6.92,0,12.652,2.596,17.193,7.787  C562.22,332.85,564.329,338.852,564.329,345.88z"
-   id="path3277" />
-<polygon
-   fill="#BA160C"
-   points="661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 656.952,339.555 591.906,339.555   591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 "
-   id="polygon3279" />
-</svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbase_logo_with_orca.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_logo_with_orca.png b/src/main/site/resources/images/hbase_logo_with_orca.png
deleted file mode 100644
index 7ed60e2..0000000
Binary files a/src/main/site/resources/images/hbase_logo_with_orca.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbase_logo_with_orca.xcf
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_logo_with_orca.xcf b/src/main/site/resources/images/hbase_logo_with_orca.xcf
deleted file mode 100644
index 8d88da2..0000000
Binary files a/src/main/site/resources/images/hbase_logo_with_orca.xcf and /dev/null differ

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbase_replication_diagram.jpg
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbase_replication_diagram.jpg b/src/main/site/resources/images/hbase_replication_diagram.jpg
deleted file mode 100644
index c110309..0000000
Binary files a/src/main/site/resources/images/hbase_replication_diagram.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbasecon2015.30percent.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbasecon2015.30percent.png b/src/main/site/resources/images/hbasecon2015.30percent.png
deleted file mode 100644
index 26896a4..0000000
Binary files a/src/main/site/resources/images/hbasecon2015.30percent.png and /dev/null differ

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbasecon2017.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbasecon2017.png b/src/main/site/resources/images/hbasecon2017.png
deleted file mode 100644
index 4b25f89..0000000
Binary files a/src/main/site/resources/images/hbasecon2017.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hbaseconasia2017.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hbaseconasia2017.png b/src/main/site/resources/images/hbaseconasia2017.png
deleted file mode 100644
index 8548870..0000000
Binary files a/src/main/site/resources/images/hbaseconasia2017.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hfile.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hfile.png b/src/main/site/resources/images/hfile.png
deleted file mode 100644
index 5762970..0000000
Binary files a/src/main/site/resources/images/hfile.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/hfilev2.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/hfilev2.png b/src/main/site/resources/images/hfilev2.png
deleted file mode 100644
index 54cc0cf..0000000
Binary files a/src/main/site/resources/images/hfilev2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/jumping-orca_rotated.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/jumping-orca_rotated.png b/src/main/site/resources/images/jumping-orca_rotated.png
deleted file mode 100644
index 4c2c72e..0000000
Binary files a/src/main/site/resources/images/jumping-orca_rotated.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/jumping-orca_rotated.xcf
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/jumping-orca_rotated.xcf b/src/main/site/resources/images/jumping-orca_rotated.xcf
deleted file mode 100644
index 01be6ff..0000000
Binary files a/src/main/site/resources/images/jumping-orca_rotated.xcf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/jumping-orca_rotated_12percent.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/jumping-orca_rotated_12percent.png b/src/main/site/resources/images/jumping-orca_rotated_12percent.png
deleted file mode 100644
index 1942f9a..0000000
Binary files a/src/main/site/resources/images/jumping-orca_rotated_12percent.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/jumping-orca_rotated_25percent.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/jumping-orca_rotated_25percent.png b/src/main/site/resources/images/jumping-orca_rotated_25percent.png
deleted file mode 100644
index 219c657..0000000
Binary files a/src/main/site/resources/images/jumping-orca_rotated_25percent.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/jumping-orca_transparent_rotated.xcf
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/jumping-orca_transparent_rotated.xcf b/src/main/site/resources/images/jumping-orca_transparent_rotated.xcf
deleted file mode 100644
index be9e3d9..0000000
Binary files a/src/main/site/resources/images/jumping-orca_transparent_rotated.xcf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/region_split_process.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/region_split_process.png b/src/main/site/resources/images/region_split_process.png
deleted file mode 100644
index 2717617..0000000
Binary files a/src/main/site/resources/images/region_split_process.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/region_states.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/region_states.png b/src/main/site/resources/images/region_states.png
deleted file mode 100644
index ba69e97..0000000
Binary files a/src/main/site/resources/images/region_states.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/replication_overview.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/replication_overview.png b/src/main/site/resources/images/replication_overview.png
deleted file mode 100644
index 47d7b4c..0000000
Binary files a/src/main/site/resources/images/replication_overview.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/images/timeline_consistency.png
----------------------------------------------------------------------
diff --git a/src/main/site/resources/images/timeline_consistency.png b/src/main/site/resources/images/timeline_consistency.png
deleted file mode 100644
index 94c47e0..0000000
Binary files a/src/main/site/resources/images/timeline_consistency.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
----------------------------------------------------------------------
diff --git a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar b/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
deleted file mode 100644
index 5b93209..0000000
Binary files a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar and /dev/null differ


[06/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/index.xml b/src/site/xdoc/index.xml
new file mode 100644
index 0000000..1848d40
--- /dev/null
+++ b/src/site/xdoc/index.xml
@@ -0,0 +1,109 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Apache HBase&#8482; Home</title>
+    <link rel="shortcut icon" href="/images/favicon.ico" />
+  </properties>
+
+  <body>
+    <section name="Welcome to Apache HBase&#8482;">
+        <p><a href="http://www.apache.org/">Apache</a> HBase&#8482; is the <a href="http://hadoop.apache.org/">Hadoop</a> database, a distributed, scalable, big data store.
+    </p>
+    <p>Use Apache HBase&#8482; when you need random, realtime read/write access to your Big Data.
+    This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware.
+Apache HBase is an open-source, distributed, versioned, non-relational database modeled after Google's <a href="http://research.google.com/archive/bigtable.html">Bigtable: A Distributed Storage System for Structured Data</a> by Chang et al.
+ Just as Bigtable leverages the distributed data storage provided by the Google File System, Apache HBase provides Bigtable-like capabilities on top of Hadoop and HDFS.
+    </p>
+  </section>
+    <section name="Download">
+    <p>
+    Click <b><a href="http://www.apache.org/dyn/closer.cgi/hbase/">here</a></b> to download Apache HBase&#8482;.
+    </p>
+    </section>
+    <section name="Features">
+    <p>
+<ul>
+    <li>Linear and modular scalability.
+</li>
+    <li>Strictly consistent reads and writes.
+</li>
+    <li>Automatic and configurable sharding of tables
+</li>
+    <li>Automatic failover support between RegionServers.
+</li>
+    <li>Convenient base classes for backing Hadoop MapReduce jobs with Apache HBase tables.
+</li>
+    <li>Easy to use Java API for client access.
+</li>
+    <li>Block cache and Bloom Filters for real-time queries.
+</li>
+    <li>Query predicate push down via server side Filters
+</li>
+    <li>Thrift gateway and a REST-ful Web service that supports XML, Protobuf, and binary data encoding options
+</li>
+    <li>Extensible jruby-based (JIRB) shell
+</li>
+    <li>Support for exporting metrics via the Hadoop metrics subsystem to files or Ganglia; or via JMX
+</li>
+</ul>
+</p>
+</section>
+     <section name="More Info">
+   <p>See the <a href="http://hbase.apache.org/book.html#arch.overview">Architecture Overview</a>, the <a href="http://hbase.apache.org/book.html#faq">Apache HBase Reference Guide FAQ</a>,
+    and the other documentation links.
+   </p>
+   <dl>
+     <dt>Export Control</dt>
+   <dd><p>The HBase distribution includes cryptographic software. See the export control notice <a href="export_control.html">here</a>
+   </p></dd>
+     <dt>Code Of Conduct</dt>
+   <dd><p>We expect participants in discussions on the HBase project mailing lists, Slack and IRC channels, and JIRA issues to abide by the Apache Software Foundation's <a href="http://apache.org/foundation/policies/conduct.html">Code of Conduct</a>. More information can be found <a href="coc.html">here</a>.
+   </p></dd>
+ </dl>
+</section>
+
+     <section name="News">
+       <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
+       <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
+       <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>
+        <p>December 8th, 2016 <a href="https://www.meetup.com/hbaseusergroup/events/235542241/">Meetup@Splice</a> in San Francisco</p>
+       <p>September 26th, 2016 <a href="http://www.meetup.com/HBase-NYC/events/233024937/">HBaseConEast2016</a> at Google in Chelsea, NYC</p>
+         <p>May 24th, 2016 <a href="http://www.hbasecon.com/">HBaseCon2016</a> at The Village, 969 Market, San Francisco</p>
+       <p>June 25th, 2015 <a href="http://www.zusaar.com/event/14057003">HBase Summer Meetup 2015</a> in Tokyo</p>
+       <p>May 7th, 2015 <a href="http://hbasecon.com/">HBaseCon2015</a> in San Francisco</p>
+       <p>February 17th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/219260093/">HBase meetup around Strata+Hadoop World</a> in San Jose</p>
+       <p>January 15th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/218744798/">HBase meetup @ AppDynamics</a> in San Francisco</p>
+       <p>November 20th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/205219992/">HBase meetup @ WANdisco</a> in San Ramon</p>
+       <p>October 27th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/207386102/">HBase Meetup @ Apple</a> in Cupertino</p>
+       <p>October 15th, 2014 <a href="http://www.meetup.com/HBase-NYC/events/207655552/">HBase Meetup @ Google</a> on the night before Strata/HW in NYC</p>
+       <p>September 25th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/203173692/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
+         <p>August 28th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/197773762/">HBase Meetup @ Sift Science</a> in San Francisco</p>
+         <p>July 17th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/190994082/">HBase Meetup @ HP</a> in Sunnyvale</p>
+         <p>June 5th, 2014 <a href="http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/">HBase BOF at Hadoop Summit</a>, San Jose Convention Center</p>
+         <p>May 5th, 2014 <a href="http://www.hbasecon.com/">HBaseCon2014</a> at the Hilton San Francisco on Union Square</p>
+         <p>March 12th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/160757912/">HBase Meetup @ Ancestry.com</a> in San Francisco</p>
+      <p><small><a href="old_news.html">Old News</a></small></p>
+    </section>
+  </body>
+
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/metrics.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/metrics.xml b/src/site/xdoc/metrics.xml
new file mode 100644
index 0000000..f3ab7d7
--- /dev/null
+++ b/src/site/xdoc/metrics.xml
@@ -0,0 +1,150 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title> 
+      Apache HBase (TM) Metrics
+    </title>
+  </properties>
+
+  <body>
+    <section name="Introduction">
+      <p>
+      Apache HBase (TM) emits Hadoop <a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html">metrics</a>.
+      </p>
+      </section>
+      <section name="Setup">
+      <p>First read up on Hadoop <a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html">metrics</a>.
+      If you are using ganglia, the <a href="http://wiki.apache.org/hadoop/GangliaMetrics">GangliaMetrics</a>
+      wiki page is useful read.</p>
+      <p>To have HBase emit metrics, edit <code>$HBASE_HOME/conf/hadoop-metrics.properties</code>
+      and enable metric 'contexts' per plugin.  As of this writing, hadoop supports
+      <strong>file</strong> and <strong>ganglia</strong> plugins.
+      Yes, the hbase metrics files is named hadoop-metrics rather than
+      <em>hbase-metrics</em> because currently at least the hadoop metrics system has the
+      properties filename hardcoded. Per metrics <em>context</em>,
+      comment out the NullContext and enable one or more plugins instead.
+      </p>
+      <p>
+      If you enable the <em>hbase</em> context, on regionservers you'll see total requests since last
+      metric emission, count of regions and storefiles as well as a count of memstore size.
+      On the master, you'll see a count of the cluster's requests.
+      </p>
+      <p>
+      Enabling the <em>rpc</em> context is good if you are interested in seeing
+      metrics on each hbase rpc method invocation (counts and time taken).
+      </p>
+      <p>
+      The <em>jvm</em> context is
+      useful for long-term stats on running hbase jvms -- memory used, thread counts, etc.
+      As of this writing, if more than one jvm is running emitting metrics, at least
+      in ganglia, the stats are aggregated rather than reported per instance.
+      </p>
+    </section>
+
+    <section name="Using with JMX">
+      <p>
+      In addition to the standard output contexts supported by the Hadoop 
+      metrics package, you can also export HBase metrics via Java Management 
+      Extensions (JMX).  This will allow viewing HBase stats in JConsole or 
+      any other JMX client.
+      </p>
+      <section name="Enable HBase stats collection">
+      <p>
+      To enable JMX support in HBase, first edit 
+      <code>$HBASE_HOME/conf/hadoop-metrics.properties</code> to support 
+      metrics refreshing. (If you've running 0.94.1 and above, or have already configured 
+      <code>hadoop-metrics.properties</code> for another output context,
+      you can skip this step).
+      </p>
+      <source>
+# Configuration of the "hbase" context for null
+hbase.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+hbase.period=60
+
+# Configuration of the "jvm" context for null
+jvm.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+jvm.period=60
+
+# Configuration of the "rpc" context for null
+rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+rpc.period=60
+      </source>
+      </section>
+      <section name="Setup JMX remote access">
+      <p>
+      For remote access, you will need to configure JMX remote passwords 
+      and access profiles.  Create the files:
+      </p>
+      <dl>
+        <dt><code>$HBASE_HOME/conf/jmxremote.passwd</code> (set permissions 
+        to 600)</dt>
+        <dd>
+        <source>
+monitorRole monitorpass
+controlRole controlpass
+        </source>
+        </dd>
+        
+        <dt><code>$HBASE_HOME/conf/jmxremote.access</code></dt>
+        <dd>
+        <source>
+monitorRole readonly
+controlRole readwrite
+        </source>
+        </dd>
+      </dl>
+      </section>
+      <section name="Configure JMX in HBase startup">
+      <p>
+      Finally, edit the <code>$HBASE_HOME/conf/hbase-env.sh</code>
+      script to add JMX support: 
+      </p>
+      <dl>
+        <dt><code>$HBASE_HOME/conf/hbase-env.sh</code></dt>
+        <dd>
+        <p>Add the lines:</p>
+        <source>
+HBASE_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false"
+HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.password.file=$HBASE_HOME/conf/jmxremote.passwd"
+HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.access.file=$HBASE_HOME/conf/jmxremote.access"
+
+export HBASE_MASTER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10101"
+export HBASE_REGIONSERVER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10102"
+        </source>
+        </dd>
+      </dl>
+      <p>
+      After restarting the processes you want to monitor, you should now be 
+      able to run JConsole (included with the JDK since JDK 5.0) to view 
+      the statistics via JMX.  HBase MBeans are exported under the 
+      <strong><code>hadoop</code></strong> domain in JMX.
+      </p>
+      </section>
+      <section name="Understanding HBase Metrics">
+      <p>
+      For more information on understanding HBase metrics, see the <a href="book.html#hbase_metrics">metrics section</a> in the Apache HBase Reference Guide. 
+      </p>
+      </section>
+    </section>
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/old_news.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/old_news.xml b/src/site/xdoc/old_news.xml
new file mode 100644
index 0000000..94e1882
--- /dev/null
+++ b/src/site/xdoc/old_news.xml
@@ -0,0 +1,92 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>
+      Old Apache HBase (TM) News
+    </title>
+  </properties>
+  <body>
+  <section name="Old News">
+         <p>February 10th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/163139322/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
+         <p>January 30th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/158491762/">HBase Meetup @ Apple</a> in Cupertino</p>
+         <p>January 30th, 2014 <a href="http://www.meetup.com/Los-Angeles-HBase-User-group/events/160560282/">Los Angeles HBase User Group</a> in El Segundo</p>
+         <p>October 24th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/140759692/">HBase User and <a href="http://www.meetup.com/hackathon/events/144366512/">Developer</a> Meetup at HortonWorks</a>.in Palo Alto</p>
+         <p>September 26, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/135862292/">HBase Meetup at Arista Networks</a>.in San Francisco</p>
+         <p>August 20th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/120534362/">HBase Meetup at Flurry</a>.in San Francisco</p>
+         <p>July 16th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/119929152/">HBase Meetup at Twitter</a>.in San Francisco</p>
+         <p>June 25th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/119154442/">Hadoop Summit Meetup</a>.at San Jose Convention Center</p>
+         <p>June 14th, 2013 <a href="http://kijicon.eventbrite.com/">KijiCon: Building Big Data Apps</a> in San Francisco.</p>
+         <p>June 13th, 2013 <a href="http://www.hbasecon.com/">HBaseCon2013</a> in San Francisco.  Submit an Abstract!</p>
+         <p>June 12th, 2013 <a href="http://www.meetup.com/hackathon/events/123403802/">HBaseConHackAthon</a> at the Cloudera office in San Francisco.</p>
+         <p>April 11th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/103587852/">HBase Meetup at AdRoll</a> in San Francisco</p>
+         <p>February 28th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/96584102/">HBase Meetup at Intel Mission Campus</a></p>
+         <p>February 19th, 2013 <a href="http://www.meetup.com/hackathon/events/103633042/">Developers PowWow</a> at HortonWorks' new digs</p>
+         <p>January 23rd, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/91381312/">HBase Meetup at WibiData World HQ!</a></p>
+            <p>December 4th, 2012 <a href="http://www.meetup.com/hackathon/events/90536432/">0.96 Bug Squashing and Testing Hackathon</a> at Cloudera, SF.</p>
+            <p>October 29th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/82791572/">HBase User Group Meetup</a> at Wize Commerce in San Mateo.</p>
+            <p>October 25th, 2012 <a href="http://www.meetup.com/HBase-NYC/events/81728932/">Strata/Hadoop World HBase Meetup.</a> in NYC</p>
+            <p>September 11th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/80621872/">Contributor's Pow-Wow at HortonWorks HQ.</a></p>
+            <p>August 8th, 2012 <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Apache HBase 0.94.1 is available for download</a></p>
+            <p>June 15th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/59829652/">Birds-of-a-feather</a> in San Jose, day after <a href="http://hadoopsummit.org">Hadoop Summit</a></p>
+            <p>May 23rd, 2012 <a href="http://www.meetup.com/hackathon/events/58953522/">HackConAthon</a> in Palo Alto</p>
+            <p>May 22nd, 2012 <a href="http://www.hbasecon.com">HBaseCon2012</a> in San Francisco</p>
+            <p>March 27th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/56021562/">Meetup @ StumbleUpon</a> in San Francisco</p>
+
+            <p>January 19th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/46702842/">Meetup @ EBay</a></p>
+            <p>January 23rd, 2012 Apache HBase 0.92.0 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
+            <p>December 23rd, 2011 Apache HBase 0.90.5 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
+            <p>November 29th, 2011 <a href="http://www.meetup.com/hackathon/events/41025972/">Developer Pow-Wow in SF</a> at Salesforce HQ</p>
+            <p>November 7th, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/35682812/">HBase Meetup in NYC (6PM)</a> at the AppNexus office</p>
+            <p>August 22nd, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/28518471/">HBase Hackathon (11AM) and Meetup (6PM)</a> at FB in PA</p>
+            <p>June 30th, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/20572251/">HBase Contributor Day</a>, the day after the <a href="http://developer.yahoo.com/events/hadoopsummit2011/">Hadoop Summit</a> hosted by Y!</p>
+            <p>June 8th, 2011 <a href="http://berlinbuzzwords.de/wiki/hbase-workshop-and-hackathon">HBase Hackathon</a> in Berlin to coincide with <a href="http://berlinbuzzwords.de/">Berlin Buzzwords</a></p>
+            <p>May 19th, 2011 Apache HBase 0.90.3 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
+            <p>April 12th, 2011 Apache HBase 0.90.2 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
+            <p>March 21st, <a href="http://www.meetup.com/hackathon/events/16770852/">HBase 0.92 Hackathon at StumbleUpon, SF</a></p>
+            <p>February 22nd, <a href="http://www.meetup.com/hbaseusergroup/events/16492913/">HUG12: February HBase User Group at StumbleUpon SF</a></p>
+            <p>December 13th, <a href="http://www.meetup.com/hackathon/calendar/15597555/">HBase Hackathon: Coprocessor Edition</a></p>
+      <p>November 19th, <a href="http://huguk.org/">Hadoop HUG in London</a> is all about Apache HBase</p>
+      <p>November 15-19th, <a href="http://www.devoxx.com/display/Devoxx2K10/Home">Devoxx</a> features HBase Training and multiple HBase presentations</p>
+      <p>October 12th, HBase-related presentations by core contributors and users at <a href="http://www.cloudera.com/company/press-center/hadoop-world-nyc/">Hadoop World 2010</a></p>
+      <p>October 11th, <a href="http://www.meetup.com/hbaseusergroup/calendar/14606174/">HUG-NYC: HBase User Group NYC Edition</a> (Night before Hadoop World)</p>
+      <p>June 30th, <a href="http://www.meetup.com/hbaseusergroup/calendar/13562846/">Apache HBase Contributor Workshop</a> (Day after Hadoop Summit)</p>
+      <p>May 10th, 2010: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project </p>
+      <p>Signup for <a href="http://www.meetup.com/hbaseusergroup/calendar/12689490/">HBase User Group Meeting, HUG10</a> hosted by Trend Micro, April 19th, 2010</p>
+
+      <p><a href="http://www.meetup.com/hbaseusergroup/calendar/12689351/">HBase User Group Meeting, HUG9</a> hosted by Mozilla, March 10th, 2010</p>
+      <p>Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/12241393/">HBase User Group Meeting, HUG8</a>, January 27th, 2010 at StumbleUpon in SF</p>
+      <p>September 8th, 2010: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release.  Get it off the <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Releases</a> page.</p>
+      <p><a href="http://dev.us.apachecon.com/c/acus2009/">ApacheCon</a> in Oakland: November 2-6th, 2009:
+      The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.</p>
+      <p>HBase at Hadoop World in NYC: October 2nd, 2009: A few of us will be talking on Practical HBase out east at <a href="http://www.cloudera.com/hadoop-world-nyc">Hadoop World: NYC</a>.</p>
+      <p>HUG7 and HBase Hackathon: August 7th-9th, 2009 at StumbleUpon in SF: Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/10950511/">HBase User Group Meeting, HUG7</a> or for the <a href="http://www.meetup.com/hackathon/calendar/10951718/">Hackathon</a> or for both (all are welcome!).</p>
+      <p>June, 2009 -- HBase at HadoopSummit2009 and at NOSQL: See the <a href="http://wiki.apache.org/hadoop/HBase/HBasePresentations">presentations</a></p>
+      <p>March 3rd, 2009 -- HUG6: <a href="http://www.meetup.com/hbaseusergroup/calendar/9764004/">HBase User Group 6</a></p>
+      <p>January 30th, 2009 -- LA Hbackathon:<a href="http://www.meetup.com/hbasela/calendar/9450876/">HBase January Hackathon Los Angeles</a> at <a href="http://streamy.com" >Streamy</a> in Manhattan Beach</p>
+  </section>
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/poweredbyhbase.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/poweredbyhbase.xml b/src/site/xdoc/poweredbyhbase.xml
new file mode 100644
index 0000000..ff1ba59
--- /dev/null
+++ b/src/site/xdoc/poweredbyhbase.xml
@@ -0,0 +1,398 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Powered By Apache HBase&#153;</title>
+  </properties>
+
+<body>
+<section name="Powered By Apache HBase&#153;">
+  <p>This page lists some institutions and projects which are using HBase. To
+    have your organization added, file a documentation JIRA or email
+    <a href="mailto:dev@hbase.apache.org">hbase-dev</a> with the relevant
+    information. If you notice out-of-date information, use the same avenues to
+    report it.
+  </p>
+  <p><b>These items are user-submitted and the HBase team assumes no responsibility for their accuracy.</b></p>
+  <dl>
+  <dt><a href="http://www.adobe.com">Adobe</a></dt>
+  <dd>We currently have about 30 nodes running HDFS, Hadoop and HBase  in clusters
+    ranging from 5 to 14 nodes on both production and development. We plan a
+    deployment on an 80 nodes cluster. We are using HBase in several areas from
+    social services to structured data and processing for internal use. We constantly
+    write data to HBase and run mapreduce jobs to process then store it back to
+    HBase or external systems. Our production cluster has been running since Oct 2008.</dd>
+
+  <dt><a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Project Astro</a></dt>
+  <dd>
+    Astro provides fast Spark SQL/DataFrame capabilities to HBase data,
+    featuring super-efficient access to multi-dimensional HBase rows through
+    native Spark execution in HBase coprocessor plus systematic and accurate
+    partition pruning and predicate pushdown from arbitrarily complex data
+    filtering logic. The batch load is optimized to run on the Spark execution
+    engine. Note that <a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Spark-SQL-on-HBase</a>
+    is the release site. Interested parties are free to make clones and claim
+    to be "latest(and active)", but they are not endorsed by the owner.
+  </dd>
+
+  <dt><a href="http://axibase.com/products/axibase-time-series-database/">Axibase
+    Time Series Database (ATSD)</a></dt>
+  <dd>ATSD runs on top of HBase to collect, analyze and visualize time series
+    data at scale. ATSD capabilities include optimized storage schema, built-in
+    rule engine, forecasting algorithms (Holt-Winters and ARIMA) and next-generation
+    graphics designed for high-frequency data. Primary use cases: IT infrastructure
+    monitoring, data consolidation, operational historian in OPC environments.</dd>
+
+  <dt><a href="http://www.benipaltechnologies.com">Benipal Technologies</a></dt>
+  <dd>We have a 35 node cluster used for HBase and Mapreduce with Lucene / SOLR
+    and katta integration to create and finetune our search databases. Currently,
+    our HBase installation has over 10 Billion rows with 100s of datapoints per row.
+    We compute over 10<sup>18</sup> calculations daily using MapReduce directly on HBase. We
+    heart HBase.</dd>
+
+  <dt><a href="https://github.com/ermanpattuk/BigSecret">BigSecret</a></dt>
+  <dd>BigSecret is a security framework that is designed to secure Key-Value data,
+    while preserving efficient processing capabilities. It achieves cell-level
+    security, using combinations of different cryptographic techniques, in an
+    efficient and secure manner. It provides a wrapper library around HBase.</dd>
+
+  <dt><a href="http://caree.rs">Caree.rs</a></dt>
+  <dd>Accelerated hiring platform for HiTech companies. We use HBase and Hadoop
+    for all aspects of our backend - job and company data storage, analytics
+    processing, machine learning algorithms for our hire recommendation engine.
+    Our live production site is directly served from HBase. We use cascading for
+    running offline data processing jobs.</dd>
+
+  <dt><a href="http://www.celer-tech.com/">Celer Technologies</a></dt>
+  <dd>Celer Technologies is a global financial software company that creates
+    modular-based systems that have the flexibility to meet tomorrow's business
+    environment, today.  The Celer framework uses Hadoop/HBase for storing all
+    financial data for trading, risk, clearing in a single data store. With our
+    flexible framework and all the data in Hadoop/HBase, clients can build new
+    features to quickly extract data based on their trading, risk and clearing
+    activities from one single location.</dd>
+
+  <dt><a href="http://www.explorys.net">Explorys</a></dt>
+  <dd>Explorys uses an HBase cluster containing over a billion anonymized clinical
+    records, to enable subscribers to search and analyze patient populations,
+    treatment protocols, and clinical outcomes.</dd>
+
+  <dt><a href="http://www.facebook.com/notes/facebook-engineering/the-underlying-technology-of-messages/454991608919">Facebook</a></dt>
+  <dd>Facebook uses HBase to power their Messages infrastructure.</dd>
+
+  <dt><a href="http://www.filmweb.pl">Filmweb</a></dt>
+  <dd>Filmweb is a film web portal with a large dataset of films, persons and
+    movie-related entities. We have just started a small cluster of 3 HBase nodes
+    to handle our web cache persistency layer. We plan to increase the cluster
+    size, and also to start migrating some of the data from our databases which
+    have some demanding scalability requirements.</dd>
+
+  <dt><a href="http://www.flurry.com">Flurry</a></dt>
+  <dd>Flurry provides mobile application analytics. We use HBase and Hadoop for
+    all of our analytics processing, and serve all of our live requests directly
+    out of HBase on our 50 node production cluster with tens of billions of rows
+    over several tables.</dd>
+
+  <dt><a href="http://gumgum.com">GumGum</a></dt>
+  <dd>GumGum is an In-Image Advertising Platform. We use HBase on an 15-node
+    Amazon EC2 High-CPU Extra Large (c1.xlarge) cluster for both real-time data
+    and analytics. Our production cluster has been running since June 2010.</dd>
+
+  <dt><a href="http://helprace.com/help-desk/">Helprace</a></dt>
+  <dd>Helprace is a customer service platform which uses Hadoop for analytics
+    and internal searching and filtering. Being on HBase we can share our HBase
+    and Hadoop cluster with other Hadoop processes - this particularly helps in
+    keeping community speeds up. We use Hadoop and HBase on small cluster with 4
+    cores and 32 GB RAM each.</dd>
+
+  <dt><a href="http://hubspot.com">HubSpot</a></dt>
+  <dd>HubSpot is an online marketing platform, providing analytics, email, and
+    segmentation of leads/contacts.  HBase is our primary datastore for our customers'
+    customer data, with multiple HBase clusters powering the majority of our
+    product.  We have nearly 200 regionservers across the various clusters, and
+    2 hadoop clusters also with nearly 200 tasktrackers.  We use c1.xlarge in EC2
+    for both, but are starting to move some of that to baremetal hardware.  We've
+    been running HBase for over 2 years.</dd>
+
+  <dt><a href="http://www.infolinks.com/">Infolinks</a></dt>
+  <dd>Infolinks is an In-Text ad provider. We use HBase to process advertisement
+    selection and user events for our In-Text ad network. The reports generated
+    from HBase are used as feedback for our production system to optimize ad
+    selection.</dd>
+
+  <dt><a href="http://www.kalooga.com">Kalooga</a></dt>
+  <dd>Kalooga is a discovery service for image galleries. We use Hadoop, HBase
+    and Pig on a 20-node cluster for our crawling, analysis and events
+    processing.</dd>
+
+  <dt><a href="http://www.leanxcale.com/">LeanXcale</a></dt>
+  <dd>LeanXcale provides an ultra-scalable transactional &amp; SQL database that
+  stores its data on HBase and it is able to scale to 1000s of nodes. It
+  also provides a standalone full ACID HBase with transactions across
+  arbitrary sets of rows and tables.</dd>
+
+
+  <dt><a href="http://www.mahalo.com">Mahalo</a></dt>
+  <dd>Mahalo, "...the world's first human-powered search engine". All the markup
+    that powers the wiki is stored in HBase. It's been in use for a few months now.
+    MediaWiki - the same software that power Wikipedia - has version/revision control.
+    Mahalo's in-house editors produce a lot of revisions per day, which was not
+    working well in a RDBMS. An hbase-based solution for this was built and tested,
+    and the data migrated out of MySQL and into HBase. Right now it's at something
+    like 6 million items in HBase. The upload tool runs every hour from a shell
+    script to back up that data, and on 6 nodes takes about 5-10 minutes to run -
+    and does not slow down production at all.</dd>
+
+  <dt><a href="http://www.meetup.com">Meetup</a></dt>
+  <dd>Meetup is on a mission to help the world’s people self-organize into local
+    groups.  We use Hadoop and HBase to power a site-wide, real-time activity
+    feed system for all of our members and groups.  Group activity is written
+    directly to HBase, and indexed per member, with the member's custom feed
+    served directly from HBase for incoming requests.  We're running HBase
+    0.20.0 on a 11 node cluster.</dd>
+
+  <dt><a href="http://www.mendeley.com">Mendeley</a></dt>
+  <dd>Mendeley is creating a platform for researchers to collaborate and share
+    their research online. HBase is helping us to create the world's largest
+    research paper collection and is being used to store all our raw imported data.
+    We use a lot of map reduce jobs to process these papers into pages displayed
+    on the site. We also use HBase with Pig to do analytics and produce the article
+    statistics shown on the web site. You can find out more about how we use HBase
+    in the <a href="http://www.slideshare.net/danharvey/hbase-at-mendeley">HBase
+    At Mendeley</a> slide presentation.</dd>
+
+  <dt><a href="http://www.ngdata.com">NGDATA</a></dt>
+  <dd>NGDATA delivers <a href="http://www.ngdata.com/site/products/lily.html">Lily</a>,
+    the consumer intelligence solution that delivers a unique combination of Big
+    Data management, machine learning technologies and consumer intelligence
+    applications in one integrated solution to allow better, and more dynamic,
+    consumer insights. Lily allows companies to process and analyze massive structured
+    and unstructured data, scale storage elastically and locate actionable data
+    quickly from large data sources in near real time.</dd>
+
+  <dt><a href="http://ning.com">Ning</a></dt>
+  <dd>Ning uses HBase to store and serve the results of processing user events
+    and log files, which allows us to provide near-real time analytics and
+    reporting. We use a small cluster of commodity machines with 4 cores and 16GB
+    of RAM per machine to handle all our analytics and reporting needs.</dd>
+
+  <dt><a href="http://www.worldcat.org">OCLC</a></dt>
+  <dd>OCLC uses HBase as the main data store for WorldCat, a union catalog which
+    aggregates the collections of 72,000 libraries in 112 countries and territories.
+    WorldCat is currently comprised of nearly 1 billion records with nearly 2
+    billion library ownership indications. We're running a 50 Node HBase cluster
+    and a separate offline map-reduce cluster.</dd>
+
+  <dt><a href="http://olex.openlogic.com">OpenLogic</a></dt>
+  <dd>OpenLogic stores all the world's Open Source packages, versions, files,
+    and lines of code in HBase for both near-real-time access and analytical
+    purposes. The production cluster has well over 100TB of disk spread across
+    nodes with 32GB+ RAM and dual-quad or dual-hex core CPU's.</dd>
+
+  <dt><a href="http://www.openplaces.org">Openplaces</a></dt>
+  <dd>Openplaces is a search engine for travel that uses HBase to store terabytes
+    of web pages and travel-related entity records (countries, cities, hotels,
+    etc.). We have dozens of MapReduce jobs that crunch data on a daily basis.
+    We use a 20-node cluster for development, a 40-node cluster for offline
+    production processing and an EC2 cluster for the live web site.</dd>
+
+  <dt><a href="http://www.pnl.gov">Pacific Northwest National Laboratory</a></dt>
+  <dd>Hadoop and HBase (Cloudera distribution) are being used within PNNL's
+    Computational Biology &amp; Bioinformatics Group for a systems biology data
+    warehouse project that integrates high throughput proteomics and transcriptomics
+    data sets coming from instruments in the Environmental  Molecular Sciences
+    Laboratory, a US Department of Energy national user facility located at PNNL.
+    The data sets are being merged and annotated with other public genomics
+    information in the data warehouse environment, with Hadoop analysis programs
+    operating on the annotated data in the HBase tables. This work is hosted by
+    <a href="http://www.pnl.gov/news/release.aspx?id=908">olympus</a>, a large PNNL
+    institutional computing cluster, with the HBase tables being stored in olympus's
+    Lustre file system.</dd>
+
+  <dt><a href="http://www.readpath.com/">ReadPath</a></dt>
+  <dd>|ReadPath uses HBase to store several hundred million RSS items and dictionary
+    for its RSS newsreader. Readpath is currently running on an 8 node cluster.</dd>
+
+  <dt><a href="http://resu.me/">resu.me</a></dt>
+  <dd>Career network for the net generation. We use HBase and Hadoop for all
+    aspects of our backend - user and resume data storage, analytics processing,
+    machine learning algorithms for our job recommendation engine. Our live
+    production site is directly served from HBase. We use cascading for running
+    offline data processing jobs.</dd>
+
+  <dt><a href="http://www.runa.com/">Runa Inc.</a></dt>
+  <dd>Runa Inc. offers a SaaS that enables online merchants to offer dynamic
+    per-consumer, per-product promotions embedded in their website. To implement
+    this we collect the click streams of all their visitors to determine along
+    with the rules of the merchant what promotion to offer the visitor at different
+    points of their browsing the Merchant website. So we have lots of data and have
+    to do lots of off-line and real-time analytics. HBase is the core for us.
+    We also use Clojure and our own open sourced distributed processing framework,
+    Swarmiji. The HBase Community has been key to our forward movement with HBase.
+    We're looking for experienced developers to join us to help make things go even
+    faster!</dd>
+
+  <dt><a href="http://www.sematext.com/">Sematext</a></dt>
+  <dd>Sematext runs
+    <a href="http://www.sematext.com/search-analytics/index.html">Search Analytics</a>,
+    a service that uses HBase to store search activity and MapReduce to produce
+    reports showing user search behaviour and experience. Sematext runs
+    <a href="http://www.sematext.com/spm/index.html">Scalable Performance Monitoring (SPM)</a>,
+    a service that uses HBase to store performance data over time, crunch it with
+    the help of MapReduce, and display it in a visually rich browser-based UI.
+    Interestingly, SPM features
+    <a href="http://www.sematext.com/spm/hbase-performance-monitoring/index.html">SPM for HBase</a>,
+    which is specifically designed to monitor all HBase performance metrics.</dd>
+
+  <dt><a href="http://www.socialmedia.com/">SocialMedia</a></dt>
+  <dd>SocialMedia uses HBase to store and process user events which allows us to
+    provide near-realtime user metrics and reporting. HBase forms the heart of
+    our Advertising Network data storage and management system. We use HBase as
+    a data source and sink for both realtime request cycle queries and as a
+    backend for mapreduce analysis.</dd>
+
+  <dt><a href="http://www.splicemachine.com/">Splice Machine</a></dt>
+  <dd>Splice Machine is built on top of HBase.  Splice Machine is a full-featured
+    ANSI SQL database that provides real-time updates, secondary indices, ACID
+    transactions, optimized joins, triggers, and UDFs.</dd>
+
+  <dt><a href="http://www.streamy.com/">Streamy</a></dt>
+  <dd>Streamy is a recently launched realtime social news site.  We use HBase
+    for all of our data storage, query, and analysis needs, replacing an existing
+    SQL-based system.  This includes hundreds of millions of documents, sparse
+    matrices, logs, and everything else once done in the relational system. We
+    perform significant in-memory caching of query results similar to a traditional
+    Memcached/SQL setup as well as other external components to perform joining
+    and sorting.  We also run thousands of daily MapReduce jobs using HBase tables
+    for log analysis, attention data processing, and feed crawling.  HBase has
+    helped us scale and distribute in ways we could not otherwise, and the
+    community has provided consistent and invaluable assistance.</dd>
+
+  <dt><a href="http://www.stumbleupon.com/">Stumbleupon</a></dt>
+  <dd>Stumbleupon and <a href="http://su.pr">Su.pr</a> use HBase as a real time
+    data storage and analytics platform. Serving directly out of HBase, various site
+    features and statistics are kept up to date in a real time fashion. We also
+    use HBase a map-reduce data source to overcome traditional query speed limits
+    in MySQL.</dd>
+
+  <dt><a href="http://www.tokenizer.org">Shopping Engine at Tokenizer</a></dt>
+  <dd>Shopping Engine at Tokenizer is a web crawler; it uses HBase to store URLs
+    and Outlinks (AnchorText + LinkedURL): more than a billion. It was initially
+    designed as Nutch-Hadoop extension, then (due to very specific 'shopping'
+    scenario) moved to SOLR + MySQL(InnoDB) (ten thousands queries per second),
+    and now - to HBase. HBase is significantly faster due to: no need for huge
+    transaction logs, column-oriented design exactly matches 'lazy' business logic,
+    data compression, !MapReduce support. Number of mutable 'indexes' (term from
+    RDBMS) significantly reduced due to the fact that each 'row::column' structure
+    is physically sorted by 'row'. MySQL InnoDB engine is best DB choice for
+    highly-concurrent updates. However, necessity to flash a block of data to
+    harddrive even if we changed only few bytes is obvious bottleneck. HBase
+    greatly helps: not-so-popular in modern DBMS 'delete-insert', 'mutable primary
+    key', and 'natural primary key' patterns become a big advantage with HBase.</dd>
+
+  <dt><a href="http://traackr.com/">Traackr</a></dt>
+  <dd>Traackr uses HBase to store and serve online influencer data in real-time.
+    We use MapReduce to frequently re-score our entire data set as we keep updating
+    influencer metrics on a daily basis.</dd>
+
+  <dt><a href="http://trendmicro.com/">Trend Micro</a></dt>
+  <dd>Trend Micro uses HBase as a foundation for cloud scale storage for a variety
+    of applications. We have been developing with HBase since version 0.1 and
+    production since version 0.20.0.</dd>
+
+  <dt><a href="http://www.twitter.com">Twitter</a></dt>
+  <dd>Twitter runs HBase across its entire Hadoop cluster. HBase provides a
+    distributed, read/write backup of all  mysql tables in Twitter's production
+    backend, allowing engineers to run MapReduce jobs over the data while maintaining
+    the ability to apply periodic row updates (something that is more difficult
+    to do with vanilla HDFS).  A number of applications including people search
+    rely on HBase internally for data generation. Additionally, the operations
+    team uses HBase as a timeseries database for cluster-wide monitoring/performance
+    data.</dd>
+
+  <dt><a href="http://www.udanax.org">Udanax.org</a></dt>
+  <dd>Udanax.org is a URL shortener which use 10 nodes HBase cluster to store URLs,
+    Web Log data and response the real-time request on its Web Server. This
+    application is now used for some twitter clients and a number of web sites.
+    Currently API requests are almost 30 per second and web redirection requests
+    are about 300 per second.</dd>
+
+  <dt><a href="http://www.veoh.com/">Veoh Networks</a></dt>
+  <dd>Veoh Networks uses HBase to store and process visitor (human) and entity
+    (non-human) profiles which are used for behavioral targeting, demographic
+    detection, and personalization services.  Our site reads this data in
+    real-time (heavily cached) and submits updates via various batch map/reduce
+    jobs. With 25 million unique visitors a month storing this data in a traditional
+    RDBMS is not an option. We currently have a 24 node Hadoop/HBase cluster and
+    our profiling system is sharing this cluster with our other Hadoop data
+    pipeline processes.</dd>
+
+  <dt><a href="http://www.videosurf.com/">VideoSurf</a></dt>
+  <dd>VideoSurf - "The video search engine that has taught computers to see".
+    We're using HBase to persist various large graphs of data and other statistics.
+    HBase was a real win for us because it let us store substantially larger
+    datasets without the need for manually partitioning the data and its
+    column-oriented nature allowed us to create schemas that were substantially
+    more efficient for storing and retrieving data.</dd>
+
+  <dt><a href="http://www.visibletechnologies.com/">Visible Technologies</a></dt>
+  <dd>Visible Technologies uses Hadoop, HBase, Katta, and more to collect, parse,
+    store, and search hundreds of millions of Social Media content. We get incredibly
+    fast throughput and very low latency on commodity hardware. HBase enables our
+    business to exist.</dd>
+
+  <dt><a href="http://www.worldlingo.com/">WorldLingo</a></dt>
+  <dd>The WorldLingo Multilingual Archive. We use HBase to store millions of
+    documents that we scan using Map/Reduce jobs to machine translate them into
+    all or selected target languages from our set of available machine translation
+    languages. We currently store 12 million documents but plan to eventually
+    reach the 450 million mark. HBase allows us to scale out as we need to grow
+    our storage capacities. Combined with Hadoop to keep the data replicated and
+    therefore fail-safe we have the backbone our service can rely on now and in
+    the future. !WorldLingo is using HBase since December 2007 and is along with
+    a few others one of the longest running HBase installation. Currently we are
+    running the latest HBase 0.20 and serving directly from it at
+    <a href="http://www.worldlingo.com/ma/enwiki/en/HBase">MultilingualArchive</a>.</dd>
+
+  <dt><a href="http://www.yahoo.com/">Yahoo!</a></dt>
+  <dd>Yahoo! uses HBase to store document fingerprint for detecting near-duplications.
+    We have a cluster of few nodes that runs HDFS, mapreduce, and HBase. The table
+    contains millions of rows. We use this for querying duplicated documents with
+    realtime traffic.</dd>
+
+  <dt><a href="http://h50146.www5.hp.com/products/software/security/icewall/eng/">HP IceWall SSO</a></dt>
+  <dd>HP IceWall SSO is a web-based single sign-on solution and uses HBase to store
+    user data to authenticate users. We have supported RDB and LDAP previously but
+    have newly supported HBase with a view to authenticate over tens of millions
+    of users and devices.</dd>
+
+  <dt><a href="http://www.ymc.ch/en/big-data-analytics-en?utm_source=hadoopwiki&amp;utm_medium=poweredbypage&amp;utm_campaign=ymc.ch">YMC AG</a></dt>
+  <dd><ul>
+    <li>operating a Cloudera Hadoop/HBase cluster for media monitoring purpose</li>
+    <li>offering technical and operative consulting for the Hadoop stack + ecosystem</li>
+    <li>editor of <a href="http://www.ymc.ch/en/hbase-split-visualisation-introducing-hannibal?utm_source=hadoopwiki&amp;utm_medium=poweredbypageamp;utm_campaign=ymc.ch">Hannibal</a>, a open-source tool
+    to visualize HBase regions sizes and splits that helps running HBase in production</li>
+  </ul></dd>
+  </dl>
+</section>
+</body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/pseudo-distributed.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/pseudo-distributed.xml b/src/site/xdoc/pseudo-distributed.xml
new file mode 100644
index 0000000..670f1e7
--- /dev/null
+++ b/src/site/xdoc/pseudo-distributed.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title> 
+Running Apache HBase (TM) in pseudo-distributed mode
+    </title>
+  </properties>
+
+  <body>
+      <p>This page has been retired.  The contents have been moved to the 
+      <a href="http://hbase.apache.org/book.html#distributed">Distributed Operation: Pseudo- and Fully-distributed modes</a> section
+ in the Reference Guide.
+ </p>
+
+ </body>
+
+</document>
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/replication.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/replication.xml b/src/site/xdoc/replication.xml
new file mode 100644
index 0000000..a2fcfcb
--- /dev/null
+++ b/src/site/xdoc/replication.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>
+      Apache HBase (TM) Replication
+    </title>
+  </properties>
+  <body>
+    <p>This information has been moved to <a href="http://hbase.apache.org/book.html#cluster_replication">the Cluster Replication</a> section of the <a href="http://hbase.apache.org/book.html">Apache HBase Reference Guide</a>.</p>
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/resources.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/resources.xml b/src/site/xdoc/resources.xml
new file mode 100644
index 0000000..19548b6
--- /dev/null
+++ b/src/site/xdoc/resources.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Other Apache HBase (TM) Resources</title>
+  </properties>
+
+<body>
+<section name="Other Apache HBase Resources">
+<section name="Books">
+<section name="HBase: The Definitive Guide">
+<p><a href="http://shop.oreilly.com/product/0636920014348.do">HBase: The Definitive Guide <i>Random Access to Your Planet-Size Data</i></a> by Lars George. Publisher: O'Reilly Media, Released: August 2011, Pages: 556.</p>
+</section>
+<section name="HBase In Action">
+<p><a href="http://www.manning.com/dimidukkhurana/">HBase In Action</a> By Nick Dimiduk and Amandeep Khurana.  Publisher: Manning, MEAP Began: January 2012, Softbound print: Fall 2012, Pages: 350.</p>
+</section>
+<section name="HBase Administration Cookbook">
+<p><a href="http://www.packtpub.com/hbase-administration-for-optimum-database-performance-cookbook/book">HBase Administration Cookbook</a> by Yifeng Jiang.  Publisher: PACKT Publishing, Release: Expected August 2012, Pages: 335.</p>
+</section>
+<section name="HBase High Performance Cookbook">
+  <p><a href="https://www.packtpub.com/big-data-and-business-intelligence/hbase-high-performance-cookbook">HBase High Performance Cookbook</a> by Ruchir Choudhry.  Publisher: PACKT Publishing, Release: January 2017, Pages: 350.</p>
+</section>
+</section>
+</section>
+</body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/sponsors.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/sponsors.xml b/src/site/xdoc/sponsors.xml
new file mode 100644
index 0000000..332f56a
--- /dev/null
+++ b/src/site/xdoc/sponsors.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Apache HBase&#153; Sponsors</title>
+  </properties>
+
+<body>
+<section name="Sponsors">
+    <p>First off, thanks to <a href="http://www.apache.org/foundation/thanks.html">all who sponsor</a>
+       our parent, the Apache Software Foundation.
+    </p>
+<p>The below companies have been gracious enough to provide their commerical tool offerings free of charge to the Apache HBase&#153; project.
+<ul>
+	<li>The crew at <a href="http://www.ej-technologies.com/">ej-technologies</a> have
+        been let us use <a href="http://www.ej-technologies.com/products/jprofiler/overview.html">JProfiler</a> for years now.</li>
+	<li>The lads at <a href="http://headwaysoftware.com/">headway software</a> have
+        given us a license for <a href="http://headwaysoftware.com/products/?code=Restructure101">Restructure101</a>
+        so we can untangle our interdependency mess.</li>
+	<li><a href="http://www.yourkit.com">YourKit</a> allows us to use their <a href="http://www.yourkit.com/overview/index.jsp">Java Profiler</a>.</li>
+	<li>Some of us use <a href="http://www.jetbrains.com/idea">IntelliJ IDEA</a> thanks to <a href="http://www.jetbrains.com/">JetBrains</a>.</li>
+  <li>Thank you to Boris at <a href="http://www.vectorportal.com/">Vector Portal</a> for granting us a license on the <a href="http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp">image</a> on which our logo is based.</li>
+</ul>
+</p>
+</section>
+<section name="Sponsoring the Apache Software Foundation">
+<p>To contribute to the Apache Software Foundation, a good idea in our opinion, see the <a href="http://www.apache.org/foundation/sponsorship.html">ASF Sponsorship</a> page.
+</p>
+</section>
+</body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/supportingprojects.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/supportingprojects.xml b/src/site/xdoc/supportingprojects.xml
new file mode 100644
index 0000000..f949a57
--- /dev/null
+++ b/src/site/xdoc/supportingprojects.xml
@@ -0,0 +1,161 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Supporting Projects</title>
+  </properties>
+
+<body>
+<section name="Supporting Projects">
+  <p>This page is a list of projects that are related to HBase. To
+    have your project added, file a documentation JIRA or email
+    <a href="mailto:dev@hbase.apache.org">hbase-dev</a> with the relevant
+    information. If you notice out-of-date information, use the same avenues to
+    report it.
+  </p>
+  <p><b>These items are user-submitted and the HBase team assumes no responsibility for their accuracy.</b></p>
+  <h3>Projects that add new features to HBase</h3>
+  <dl>
+   <dt><a href="https://github.com/XiaoMi/themis/">Themis</a></dt>
+   <dd>Themis provides cross-row/cross-table transaction on HBase based on
+    Google's Percolator.</dd>
+   <dt><a href="https://github.com/caskdata/tephra">Tephra</a></dt>
+   <dd>Cask Tephra provides globally consistent transactions on top of Apache
+    HBase.</dd>
+   <dt><a href="https://github.com/VCNC/haeinsa">Haeinsa</a></dt>
+   <dd>Haeinsa is linearly scalable multi-row, multi-table transaction library
+    for HBase.</dd>
+   <dt><a href="https://github.com/juwi/HBase-TAggregator">HBase TAggregator</a></dt>
+   <dd>An HBase coprocessor for timeseries-based aggregations.</dd>
+   <dt><a href="http://trafodion.incubator.apache.org/">Apache Trafodion</a></dt>
+   <dd>Apache Trafodion is a webscale SQL-on-Hadoop solution enabling
+    transactional or operational workloads on Hadoop.</dd>
+   <dt><a href="http://phoenix.apache.org/">Apache Phoenix</a></dt>
+   <dd>Apache Phoenix is a relational database layer over HBase delivered as a
+    client-embedded JDBC driver targeting low latency queries over HBase data.</dd>
+   <dt><a href="https://github.com/cloudera/hue/tree/master/apps/hbase">Hue HBase Browser</a></dt>
+   <dd>An Easy &amp; Powerful WebUI for HBase, distributed with <a href="https://www.gethue.com">Hue</a>.</dd>
+   <dt><a href="https://github.com/NGDATA/hbase-indexer/tree/master/hbase-sep">HBase SEP</a></dt>
+   <dd>the HBase Side Effect Processor, a system for asynchronously and reliably listening to HBase
+    mutation events, based on HBase replication.</dd>
+   <dt><a href="https://github.com/ngdata/hbase-indexer">Lily HBase Indexer</a></dt>
+   <dd>indexes HBase content to Solr by listening to the replication stream
+    (uses the HBase SEP).</dd>
+   <dt><a href="https://github.com/sonalgoyal/crux/">Crux</a></dt>
+   <dd> - HBase Reporting and Analysis with support for simple and composite keys,
+    get and range scans, column based filtering, charting.</dd>
+   <dt><a href="https://github.com/yahoo/omid/">Omid</a></dt>
+   <dd> - Lock-free transactional support on top of HBase providing Snapshot
+    Isolation.</dd>
+   <dt><a href="http://dev.tailsweep.com/projects/parhely">Parhely</a></dt>
+   <dd>ORM for HBase</dd>
+   <dt><a href="http://code.google.com/p/hbase-writer/">HBase-Writer</a></dt>
+   <dd> Heritrix2 Processor for writing crawls to HBase.</dd>
+   <dt><a href="http://www.pigi-project.org/">Pigi Project</a></dt>
+   <dd>The Pigi Project is an ORM-like framework. It includes a configurable
+    index system and a simple object to HBase mapping framework (or indexing for
+    HBase if you like).  Designed for use by web applications.</dd>
+   <dt><a href="http://code.google.com/p/hbase-thrift/">hbase-thrift</a></dt>
+   <dd>hbase-thrift generates and installs Perl and Python Thrift bindings for
+    HBase.</dd>
+   <dt><a href="http://belowdeck.kissintelligentsystems.com/ohm">OHM</a></dt>
+   <dd>OHM is a weakly relational ORM for HBase which provides Object Mapping and
+    Column indexing. It has its own compiler capable of generating interface
+    code for multiple languages. Currently C# (via the Thrift API), with support
+    for Java currently in development. The compiler is easily extensible to add
+    support for other languages.</dd>
+   <dt><a href="http://datastore.googlecode.com">datastore</a></dt>
+   <dd>Aims to be an implementation of the
+    <a href="http://code.google.com/appengine/docs/python/datastore/">Google app-engine datastore</a>
+    in Java using HBase instead of bigtable.</dd>
+   <dt><a href="http://datanucleus.org">DataNucleus</a></dt>
+   <dd>DataNucleus is a Java JDO/JPA/REST implementation. It supports HBase and
+    many other datastores.</dd>
+   <dt><a href="http://github.com/impetus-opensource/Kundera">Kundera</a></dt>
+   <dd>Kundera is a JPA 2.0 based object-datastore mapping library for HBase,
+    Cassandra and MongoDB.</dd>
+   <dt><a href="http://github.com/zohmg/zohmg/tree/master">Zohmg</a></dt>
+   <dd>Zohmg is a time-series data store that uses HBase as its backing store.</dd>
+   <dt><a href="http://grails.org/plugin/gorm-hbase">Grails Support</a></dt>
+   <dd>Grails HBase plug-in.</dd>
+   <dt><a href="http://www.bigrecord.org">BigRecord</a></dt>
+   <dd>is an active_record-based object mapping layer for Ruby on Rails.</dd>
+   <dt><a href="http://github.com/greglu/hbase-stargate">hbase-stargate</a></dt>
+   <dd>Ruby client for HBase Stargate.</dd>
+   <dt><a href="http://github.com/ghelmling/meetup.beeno">Meetup.Beeno</a></dt>
+   <dd>Meetup.Beeno is a simple HBase Java "beans" mapping framework based on
+    annotations. It includes a rudimentary high level query API that generates
+    the appropriate server-side filters.</dd>
+   <dt><a href="http://www.springsource.org/spring-data/hadoop">Spring Hadoop</a></dt>
+   <dd> - The Spring Hadoop project provides support for writing Apache Hadoop
+    applications that benefit from the features of Spring, Spring Batch and
+    Spring Integration.</dd>
+   <dt><a href="https://jira.springsource.org/browse/SPR-5950">Spring Framework HBase Template</a></dt>
+   <dd>Spring Framework HBase Template provides HBase data access templates
+    similar to what is provided in Spring for JDBC, Hibernate, iBatis, etc.
+    If you find this useful, please vote for its inclusion in the Spring Framework.</dd>
+   <dt><a href="http://github.com/davidsantiago/clojure-hbase">Clojure-HBase</a></dt>
+   <dd>A library for convenient access to HBase from Clojure.</dd>
+   <dt><a href="http://www.lilyproject.org/lily/about/playground/hbaseindexes.html">HBase indexing library</a></dt>
+   <dd>A library for building and querying HBase-table-based indexes.</dd>
+   <dt><a href="http://github.com/akkumar/hbasene">HBasene</a></dt>
+   <dd>Lucene+HBase - Using HBase as the backing store for the TF-IDF
+    representations needed by Lucene. Also, contains a library for constructing
+    lucene indices from HBase schema.</dd>
+   <dt><a href="http://github.com/larsgeorge/jmxtoolkit">JMXToolkit</a></dt>
+   <dd>A HBase tailored JMX toolkit enabling monitoring with Cacti and checking
+    with Nagios or similar.</dd>
+   <dt><a href="http://github.com/ykulbak/ihbase">IHBASE</a></dt>
+   <dd>IHBASE provides faster scans by indexing regions, each region has its own
+    index. The indexed columns are user-defined and indexes can be intersected or
+    joined in a single query.</dd>
+   <dt><a href="http://github.com/apurtell/hbase-ec2">HBASE EC2 scripts</a></dt>
+   <dd>This collection of bash scripts allows you to run HBase clusters on
+    Amazon's Elastic Compute Cloud (EC2) service with best practices baked in.</dd>
+   <dt><a href="http://github.com/apurtell/hbase-stargate">Stargate</a></dt>
+   <dd>Stargate provides an enhanced RESTful interface.</dd>
+   <dt><a href="http://github.com/hbase-trx/hbase-transactional-tableindexed">HBase-trx</a></dt>
+   <dd>HBase-trx provides Transactional (JTA) and indexed extensions of HBase.</dd>
+   <dt><a href="http://github.com/simplegeo/python-hbase-thrift">HBase Thrift Python client Debian package</a></dt>
+   <dd>Debian packages for the HBase Thrift Python client (see readme for
+    sources.list setup)</dd>
+   <dt><a href="http://github.com/amitrathore/capjure">capjure</a></dt>
+   <dd>capjure is a persistence helper for HBase. It is written in the Clojure
+    language, and supports persisting of native hash-maps.</dd>
+   <dt><a href="http://github.com/sematext/HBaseHUT">HBaseHUT</a></dt>
+   <dd>(High Update Throughput for HBase) It focuses on write performance during
+    records update (by avoiding doing Get on every Put to update record).</dd>
+   <dt><a href="http://github.com/sematext/HBaseWD">HBaseWD</a></dt>
+   <dd>HBase Writes Distributor spreads records over the cluster even when their
+    keys are sequential, while still allowing fast range scans over them</dd>
+   <dt><a href="http://code.google.com/p/hbase-jdo/">HBase UI Tool &amp; Util</a></dt>
+   <dd>HBase UI Tool &amp; Util is an HBase UI client and simple util module.
+    It can handle hbase more easily like jdo(not persistence api)</dd>
+  </dl>
+  <h3>Example HBase Applications</h3>
+  <ul>
+    <li><a href="http://github.com/andreisavu/feedaggregator">HBase powered feed aggregator</a>
+    by Savu Andrei -- 200909</li>
+  </ul>
+</section>
+</body>
+</document>

[04/27] hbase git commit: HBASE-18020 Update API Compliance Checker to Incorporate Improvements Done in Hadoop

Posted by bu...@apache.org.
HBASE-18020 Update API Compliance Checker to Incorporate Improvements Done in Hadoop

- Converted to python
- Updated to most recent release of Java ACC
- Added ability to include known number of problems

Author: Andrew Wang
Amending-Author: Alex Leblang
Amending-Author: Sean Busbey

Signed-off-by: Dima Spivak <di...@apache.org>
Signed-off-by: Sean Busbey <bu...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 2bde7a1077bfe8b0e2057202204a45310b623df0
Parents: 6266bb3
Author: Alex Leblang <al...@cloudera.com>
Authored: Mon Jun 26 16:42:39 2017 -0700
Committer: Sean Busbey <bu...@apache.org>
Committed: Fri Aug 4 12:05:10 2017 -0500

----------------------------------------------------------------------
 dev-support/check_compatibility.sh | 298 ------------------
 dev-support/checkcompatibility.py  | 514 ++++++++++++++++++++++++++++++++
 2 files changed, 514 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2bde7a10/dev-support/check_compatibility.sh
----------------------------------------------------------------------
diff --git a/dev-support/check_compatibility.sh b/dev-support/check_compatibility.sh
deleted file mode 100755
index d7ae6b5..0000000
--- a/dev-support/check_compatibility.sh
+++ /dev/null
@@ -1,298 +0,0 @@
-#!/usr/bin/env bash
-#
-# 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.
-#
-#
-# check_compatibility.sh
-# A script that uses the Java API Compliance Checker (Java ACC) to gauge the binary and source
-# compatibility of two arbitrary versions of Apache HBase.
-#
-# Special thanks to Andrey Ponomarenko, the leader of the Java ACC project, for introducing
-# support for class annotation filtering into the tool at our request.
-#
-# Usage: This script checks out two versions of HBase (via a tag, branch, or commit hash in Git),
-#        builds the releases, and generates XML descriptors of relevant JARs (i.e. excluding
-#        test JARs, as well as external HBase dependencies). Next, the Java API Compliance
-#        Checker (http://ispras.linuxbase.org/index.php/Java_API_Compliance_Checker) is
-#        downloaded and run using these XML descriptor files to generate a report
-#        of the degree of binary and source compatibility of the two HBase versions. Finally,
-#        the resulting report is scraped and some of its results output to stdout.
-#
-#        Note that GNU getopt is required for this script to work properly. If you're running
-#        a different variant (e.g. OS X ships with BSD getopt), you need to get the GNU variant
-#        and either put it on the PATH or set GETOPT to the location of GNU getopt before
-#        executing this script.
-#
-# Example: To compare the binary and source compatibility of the 0.98.6 release and the
-#          tip of the master branch:
-#          $ ./check_compatibility.sh 0.98.6
-#          (i.e. if -b is omitted, a check is implicitly run against master).
-#
-#          To compare the binary and source compatibility of the HBase 0.98.5 and 0.98.6
-#          releases:
-#          $ ./check_compatibility.sh 0.98.5 0.98.6
-
-SCRIPT_DIRECTORY=$(dirname ${BASH_SOURCE[0]})
-# Save the InterfaceAudience package name as a variable to make annotation listing more convenient.
-IA_PACKAGE="org.apache.hadoop.hbase.classification.InterfaceAudience"
-
-# Usage message.
-usage () {
-  SCRIPT=$(basename "${BASH_SOURCE}")
-
-  cat << __EOF
-
-check_compatibility.sh
-A script that uses the Java API Compliance Checker to gauge the binary and source
-compatibility of two arbitrary versions of Apache HBase.
-
-Usage: [<options>] <ref1> [<ref2>]
-
-The positional arguments are Git references; this can be a tag (e.g. 0.98.6),
-a branch (e.g. 0.98), or a particular commit hash. If ref2 is omitted, master
-will be used.
-
-Options:
-  -a, --all                             Do not filter by interface annotations.
-  -b, --binary-only                     Only run the check for binary compatibility.
-  -f, --force-download                  Download dependencies (i.e. Java ACC), even if they are
-                                        already present.
-  -h, --help                            Show this screen.
-  -j, --java-acc                        Specify which version of Java ACC to use to run the
-                                        analysis. This can be a tag, branch, or commit hash.
-                                        Defaults to master.
-  -l <list>, --annotation-list=<list>   A comma-separated list of annotations to limit compatibility
-                                        checks to. Defaults to
-                                        "${IA_PACKAGE}.Public,${IA_PACKAGE}.LimitedPrivate".
-  -n, --no-checkout                     Run the tool without first using Git to checkout the two
-                                        HBase versions. If this option is selected,
-                                        dev-support/target/compatibility/1 and
-                                        dev-support/target compatibility/2 must each be Git
-                                        repositories. Also note that the references must still be
-                                        specified as these are used when naming the compatibility
-                                        report.
-  -o <opts>, --options=<opts>           A comma-separated list of options to pass directly to Java
-                                        ACC.
-  -q, --quick                           Runs Java ACC in quick analysis mode, which disables a
-                                        number of checks for things that may break compatibility.
-  -r <url>, --repo=<url>                URL of the HBase Git repository to use. Defaults to Apache
-                                        HBase's GitHub (https://github.com/apache/hbase.git).
-  -s, --source-only                     Only run the check for source compatibility.
-__EOF
-}
-
-# Allow a user to override which GETOPT to use, as described in the header.
-GETOPT=${GETOPT:-/usr/bin/env getopt}
-
-# Parse command line arguments. We split long options (-l) to stay under 100 chars.
-if ! ARG_LIST=$(${GETOPT} -q -o abfhj:nl:o:qr:s \
-    -l all,annotation-list:,binary-only,force-download,help \
-    -l java-acc:,no-checkout,options:,quick,repo:,source-only \
-    -- "${@}"); then
-  usage >&2
-  exit 2
-fi
-eval set -- "${ARG_LIST[@]}"
-
-# Set defaults for options in case they're not specified on the command line.
-ANNOTATION_LIST=(${IA_PACKAGE}.Public ${IA_PACKAGE}.LimitedPrivate)
-JAVA_ACC_COMMIT="master"
-REPO_URL="https://github.com/apache/hbase.git"
-
-while ((${#})); do
-  case "${1}" in
-    -a | --all )
-      ALL=true
-      shift 1 ;;
-    -b | --binary-only )
-      JAVA_ACC_COMMAND+=(-binary)
-      shift 1 ;;
-    -f | --force-download )
-      FORCE_DOWNLOAD=true
-      shift 1 ;;
-    -h | --help )
-      usage
-      exit 0 ;;
-    -j | --java-acc )
-      JAVA_ACC_COMMIT="${2}"
-      shift 2 ;;
-    -l | --annotation-list )
-      # Process the comma-separated list of annotations and overwrite the default list.
-      ANNOTATION_LIST=($(tr "," "\n" <<< "${2}"))
-      shift 2 ;;
-    -n | --no-checkout )
-      NO_CHECKOUT=true
-      shift 1 ;;
-    -q | --quick )
-      JAVA_ACC_COMMAND+=(-quick)
-      shift 1 ;;
-    -o | --options )
-      # Process and append the comma-separated list of options into the command array.
-      JAVA_ACC_COMMAND+=($(tr "," "\n" <<< "${2}"))
-      shift 2 ;;
-    -r | --repo )
-      REPO_URL="${2}"
-      shift 2 ;;
-    -s | --source-only )
-      JAVA_ACC_COMMAND+=(-source)
-      shift 1 ;;
-    # getopt inserts -- to separate options and positional arguments.
-    -- )
-      # First, shift past the -- to get to the positional arguments.
-      shift 1
-      # If there is one positional argument, only <ref1> was specified.
-      if [ ${#} -eq 1 ]; then
-        COMMIT[1]="${1}"
-        COMMIT[2]=master
-        shift 1
-      # If there are two positional arguments, <ref1> and <ref2> were both specified.
-      elif [ ${#} -eq 2 ]; then
-        COMMIT[1]="${1}"
-        COMMIT[2]="${2}"
-        shift 2
-      # If there are no positional arguments or too many, someone needs to reread the usage
-      # message.
-      else
-        usage >&2
-        exit 2
-      fi
-      ;;
-  esac
-done
-
-# Do identical operations for both HBase versions in a for loop to save some lines of code.
-for ref in 1 2; do
-  if ! [ "${NO_CHECKOUT}" ]; then
-    # Create empty directories for both versions in question.
-    echo "Creating empty ${SCRIPT_DIRECTORY}/target/compatibility/${ref} directory..."
-    rm -rf ${SCRIPT_DIRECTORY}/target/compatibility/${ref}
-    mkdir -p ${SCRIPT_DIRECTORY}/target/compatibility/${ref}
-
-    if [ "${ref}" = "1" ]; then
-      echo "Cloning ${REPO_URL} into ${SCRIPT_DIRECTORY}/target/compatibility/${ref}..."
-      if ! git clone ${REPO_URL} ${SCRIPT_DIRECTORY}/target/compatibility/${ref}; then
-        echo "Error while cloning ${REPO_URL}. Exiting..." >&2
-        exit 2
-      fi
-    elif [ "${ref}" = "2" ]; then
-      # Avoid cloning from Git twice by copying first repo into different folder.
-      echo "Copying Git repository into ${SCRIPT_DIRECTORY}/target/compatibility/${ref}..."
-      cp -a ${SCRIPT_DIRECTORY}/target/compatibility/1/.git \
-          ${SCRIPT_DIRECTORY}/target/compatibility/2
-    fi
-
-    # Use pushd and popd to keep track of directories while navigating around (and hide
-    # printing of the stack).
-    pushd ${SCRIPT_DIRECTORY}/target/compatibility/${ref} > /dev/null
-    echo "Checking out ${COMMIT[${ref}]} into ${ref}/..."
-    if ! git checkout -f ${COMMIT[${ref}]}; then
-      echo "Error while checking out ${COMMIT[${ref}]}. Exiting..." >&2
-      exit 2
-    fi
-    echo "Building ${COMMIT[${ref}]}..."
-    if ! mvn clean package --batch-mode -DskipTests; then
-      echo "Maven could not successfully package ${COMMIT[${ref}]}. Exiting..." >&2
-      exit 2
-    fi
-    # grab sha for future reference
-    SHA[${ref}]=$(git rev-parse --short HEAD)
-    popd > /dev/null
-  fi
-
-  JAR_FIND_EXPRESSION=(-name "hbase*.jar" ! -name "*tests*" ! -name "*sources*" ! -name "*shade*")
-  # Create an array of all the HBase JARs matching the find expression.
-  JARS=$(find ${SCRIPT_DIRECTORY}/target/compatibility/${ref} "${JAR_FIND_EXPRESSION[@]}")
-
-  if [ ${#JARS[@]} -eq 0 ]; then
-    # If --no-checkout was specified and no JARs were found, try running mvn package
-    # for the user before failing.
-    if [ ${NO_CHECKOUT} ]; then
-      for ref in 1 2; do
-        pushd ${SCRIPT_DIRECTORY}/target/compatibility/${ref} > /dev/null
-        echo "The --no-checkout option was specified, but no JARs were found." \
-            "Attempting to build ${COMMIT[${ref}]}..."
-        if ! mvn clean package --batch-mode -DskipTests; then
-          echo "Maven could not successfully package ${COMMIT[${ref}]}. Exiting..." >&2
-          exit 2
-        fi
-        SHA[${ref}]=$(git rev-parse --short HEAD)
-        popd > /dev/null
-      done
-
-      JARS=$(find ${SCRIPT_DIRECTORY}/target/compatibility/${ref} "${JAR_FIND_EXPRESSION[@]}")
-      if [ ${#JARS[@]} -eq 0 ]; then
-        echo "Unable to find any JARs matching the find expression. Exiting..." >&2
-        exit 2
-      fi
-
-    # If no JARs were found and --no-checkout was not specified, fail immediately.
-    else
-      echo "Unable to find any JARs matching the find expression. Exiting..." >&2
-    fi
-  fi
-
-  # Create an XML descriptor containing paths to the JARs for Java ACC to use (support for
-  # comma-separated lists of JARs was removed, as described on their issue tracker:
-  # https://github.com/lvc/japi-compliance-checker/issues/27).
-  DESCRIPTOR_PATH="${SCRIPT_DIRECTORY}/target/compatibility/${ref}.xml"
-  echo "<version>${COMMIT[${ref}]}${SHA[${ref}]+"/${SHA[${ref}]}"}</version>" > "${DESCRIPTOR_PATH}"
-  echo "<archives>" >> "${DESCRIPTOR_PATH}"
-
-  echo "The JARs to be analyzed from ${COMMIT[${ref}]} are:"
-  for jar in ${JARS}; do
-    echo "  ${jar}" | tee -a "${DESCRIPTOR_PATH}"
-  done
-  echo "</archives>" >> "${DESCRIPTOR_PATH}"
-done
-
-# Download the Java API Compliance Checker (Java ACC) into /dev-support/target/compatibility.
-# Note: Java API Compliance Checker (Java ACC) is licensed under the GNU GPL or LGPL. For more
-#       information, visit http://ispras.linuxbase.org/index.php/Java_API_Compliance_Checker .
-
-# Only clone Java ACC if it's missing or if option to force dependency download is present.
-if [ ! -d ${SCRIPT_DIRECTORY}/target/compatibility/javaACC ] || [ -n "${FORCE_DOWNLOAD}" ]; then
-  echo "Downloading Java API Compliance Checker..."
-  rm -rf ${SCRIPT_DIRECTORY}/target/compatibility/javaACC
-  if ! git clone https://github.com/lvc/japi-compliance-checker.git -b "${JAVA_ACC_COMMIT}" \
-      ${SCRIPT_DIRECTORY}/target/compatibility/javaACC; then
-    echo "Failed to download Java API Compliance Checker. Exiting..." >&2
-    exit 2
-  fi
-fi
-
-# Generate one-per-line list of annotations.
-tr " " "\n" <<< "${ANNOTATION_LIST[@]}" > "${SCRIPT_DIRECTORY}/target/compatibility/annotations"
-
-# Generate command line arguments for Java ACC.
-JAVA_ACC_COMMAND+=(-l HBase)
-JAVA_ACC_COMMAND+=(-old "${SCRIPT_DIRECTORY}/target/compatibility/1.xml")
-JAVA_ACC_COMMAND+=(-new "${SCRIPT_DIRECTORY}/target/compatibility/2.xml")
-JAVA_ACC_COMMAND+=(-report-path \
-    ${SCRIPT_DIRECTORY}/target/compatibility/report/${COMMIT[1]}_${COMMIT[2]}_compat_report.html)
-if [ "${ALL}" != "true" ] ; then
-  JAVA_ACC_COMMAND+=(-annotations-list ${SCRIPT_DIRECTORY}/target/compatibility/annotations)
-fi
-
-# Delete any existing report folder under /dev-support/target/compatibility.
-rm -rf ${SCRIPT_DIRECTORY}/target/compatibility/report
-
-# Run the tool. Note that Java ACC returns an exit code of 0 if the two versions are
-# compatible, an exit code of 1 if the two versions are not, and several other codes
-# for various errors. See the tool's website for details.
-echo "Running the Java API Compliance Checker..."
-perl "${SCRIPT_DIRECTORY}/target/compatibility/javaACC/japi-compliance-checker.pl" \
-    "${JAVA_ACC_COMMAND[@]}"

http://git-wip-us.apache.org/repos/asf/hbase/blob/2bde7a10/dev-support/checkcompatibility.py
----------------------------------------------------------------------
diff --git a/dev-support/checkcompatibility.py b/dev-support/checkcompatibility.py
new file mode 100755
index 0000000..0f90207
--- /dev/null
+++ b/dev-support/checkcompatibility.py
@@ -0,0 +1,514 @@
+#!/usr/bin/env python
+#
+# 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.
+
+# Script which checks Java API compatibility between two revisions of the
+# Java client.
+#
+# Originally sourced from Apache Kudu, which was based on the
+# compatibility checker from the Apache HBase project, but ported to
+# Python for better readability.
+
+# The script can be invoked as follows:
+#   $ ./checkcompatibility.py ${SOURCE_GIT_REVISION} ${GIT_BRANCH_OR_TAG}
+# or with some options:
+#   $ ./dev-support/checkcompatibility.py \
+#      --annotation org.apache.hadoop.hbase.classification.InterfaceAudience.Public \
+#      --annotation org.apache.hadoop.hbase.classification.InterfaceAudience.LimitedPrivate \
+#      --include-file "hbase-*" \
+#      --known_problems_path ~/known_problems.json \
+#      rel/1.0.0 branch-1.2
+
+import json
+import logging
+import os
+import re
+import shutil
+import subprocess
+import sys
+import urllib2
+from collections import namedtuple
+try:
+    import argparse
+except ImportError:
+    logging.error(
+        "Please install argparse, e.g. via `pip install argparse`.")
+    sys.exit(2)
+
+# Various relative paths
+REPO_DIR = os.getcwd()
+
+
+def check_output(*popenargs, **kwargs):
+    """ Run command with arguments and return its output as a byte string.
+    Backported from Python 2.7 as it's implemented as pure python on stdlib.
+    >>> check_output(['/usr/bin/python', '--version'])
+    Python 2.6.2 """
+    process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs)
+    output, _ = process.communicate()
+    retcode = process.poll()
+    if retcode:
+        cmd = kwargs.get("args")
+        if cmd is None:
+            cmd = popenargs[0]
+        error = subprocess.CalledProcessError(retcode, cmd)
+        error.output = output
+        raise error
+    return output
+
+
+def get_repo_dir():
+    """ Return the path to the top of the repo. """
+    dirname, _ = os.path.split(os.path.abspath(__file__))
+    dirname = os.path.dirname(dirname)
+    logging.debug("Repo dir is  %s", dirname)
+    return dirname
+
+
+def get_scratch_dir():
+    """ Return the path to the scratch dir that we build within. """
+    scratch_dir = os.path.join(get_repo_dir(), "target", "compat-check")
+    if not os.path.exists(scratch_dir):
+        os.makedirs(scratch_dir)
+    return scratch_dir
+
+
+def get_java_acc_dir():
+    """ Return the path where we check out the Java API Compliance Checker. """
+    return os.path.join(get_repo_dir(), "target", "java-acc")
+
+
+def clean_scratch_dir(scratch_dir):
+    """ Clean up and re-create the scratch directory. """
+    if os.path.exists(scratch_dir):
+        logging.info("Removing scratch dir %s ", scratch_dir)
+        shutil.rmtree(scratch_dir)
+    logging.info("Creating empty scratch dir %s ", scratch_dir)
+    os.makedirs(scratch_dir)
+
+
+def checkout_java_tree(rev, path):
+    """ Check out the Java source tree for the given revision into
+    the given path. """
+    logging.info("Checking out %s in %s", rev, path)
+    os.makedirs(path)
+    # Extract java source
+    subprocess.check_call(["bash", '-o', 'pipefail', "-c",
+                           ("git archive --format=tar %s | "
+                            "tar -C '%s' -xf -") % (rev, path)],
+                          cwd=get_repo_dir())
+
+
+def get_git_hash(revname):
+    """ Convert 'revname' to its SHA-1 hash. """
+    return check_output(["git", "rev-parse", revname],
+                        cwd=get_repo_dir()).strip()
+
+
+def get_repo_name():
+    """ Get the name of the repo based on the git remote."""
+    remote = check_output(["git", "config", "--get", "remote.origin.url"],
+                           cwd=get_repo_dir()).strip()
+    remote = remote.split("/")[-1]
+    return remote[:-4] if remote.endswith(".git") else remote
+
+
+def build_tree(java_path, verbose):
+    """ Run the Java build within 'path'. """
+    logging.info("Building in %s ", java_path)
+    mvn_cmd = ["mvn", "--batch-mode", "-DskipTests",
+               "-Dmaven.javadoc.skip=true", "package"]
+    if not verbose:
+        mvn_cmd.insert(-1, "--quiet")
+    subprocess.check_call(mvn_cmd, cwd=java_path)
+
+
+def checkout_java_acc(force):
+    """ Check out the Java API Compliance Checker. If 'force' is true, will
+    re-download even if the directory exists. """
+    acc_dir = get_java_acc_dir()
+    if os.path.exists(acc_dir):
+        logging.info("Java ACC is already downloaded.")
+        if not force:
+            return
+        logging.info("Forcing re-download.")
+        shutil.rmtree(acc_dir)
+
+    logging.info("Downloading Java ACC...")
+
+    url = "https://github.com/lvc/japi-compliance-checker/archive/2.1.tar.gz"
+    scratch_dir = get_scratch_dir()
+    path = os.path.join(scratch_dir, os.path.basename(url))
+    jacc = urllib2.urlopen(url)
+    with open(path, 'wb') as w:
+        w.write(jacc.read())
+
+    subprocess.check_call(["tar", "xzf", path],
+                          cwd=scratch_dir)
+
+    shutil.move(os.path.join(scratch_dir, "japi-compliance-checker-2.1"),
+                os.path.join(acc_dir))
+
+
+def find_jars(path):
+    """ Return a list of jars within 'path' to be checked for compatibility. """
+    all_jars = set(check_output(["find", path, "-name", "*.jar"]).splitlines())
+
+    return [j for j in all_jars if (
+        "-tests" not in j and
+        "-sources" not in j and
+        "-with-dependencies" not in j)]
+
+
+def write_xml_file(path, version, jars):
+    """ Write the XML manifest file for JACC. """
+    with open(path, "wt") as f:
+        f.write("<version>%s</version>\n" % version)
+        f.write("<archives>")
+        for j in jars:
+            f.write("%s\n" % j)
+        f.write("</archives>")
+
+
+def ascii_encode_dict(data):
+    """ Iterate through a dictionary of data and convert all unicode to ascii.
+    This method was taken from
+    stackoverflow.com/questions/9590382/forcing-python-json-module-to-work-with-ascii """
+    ascii_encode = lambda x: x.encode('ascii') if isinstance(x, unicode) else x
+    return dict(map(ascii_encode, pair) for pair in data.items())
+
+
+def process_json(path):
+    """ Process the known problems json file. The program raises an uncaught exception
+    if it can't find the file or if the json is invalid """
+    path = os.path.abspath(os.path.expanduser(path))
+    try:
+        with open(path) as f:
+            return json.load(f, object_hook=ascii_encode_dict)
+    except ValueError as e:
+        logging.error("File: %s\nInvalid JSON:\n%s", str(path), str(e))
+        raise
+    except IOError as io:
+        logging.error("Provided json file path does not exist %s", str(path))
+        raise
+
+
+def compare_results(tool_results, known_issues, compare_warnings):
+    """ Compare the number of problems found with the allowed number. If
+    compare_warnings is true then also compare the number of warnings found.
+
+    tool_results = results from the JACC tool - a dictionary
+    known_issues = dictionary of expected issue count
+    compare_warnings = boolean - if true also compare warnings as well as problems """
+    logging.info("Results: %s", str(tool_results))
+
+    unexpected_issue = namedtuple('unexpected_issue', ['check', 'issue_type',
+                                                        'known_count', 'observed_count'])
+    unexpected_issues = [unexpected_issue(check=check,  issue_type=issue_type,
+                                      known_count=known_count,
+                                      observed_count=tool_results[check][issue_type])
+                     for check, known_issue_counts in known_issues.items()
+                        for issue_type, known_count in known_issue_counts.items()
+                            if tool_results[check][issue_type] > known_count]
+
+    if not compare_warnings:
+        unexpected_issues = [tup for tup in unexpected_issues
+                             if tup.issue_type != 'warnings']
+
+    for issue in unexpected_issues:
+        logging.error('Found %s during  %s check (known issues: %d, observed issues: %d)',
+                issue.issue_type, issue.check, issue.known_count, issue.observed_count)
+
+    return bool(unexpected_issues)
+
+
+def process_java_acc_output(output):
+    """ Process the output string to find the problems and warnings in both the
+    binary and source compatibility. This is done in a way that is admittedly
+    brittle; we are open to better implementations.
+
+    We expect a line containing the relevant information to look something like:
+    "total binary compatibility problems: 123, warnings: 16" """
+    return_value = {}
+    output = output.split("\n")
+    for line in output:
+        # Line has relevant info
+        if line.lower().startswith("total"):
+            values = {}
+            # Remove "total" keyword
+            line = line[6:]
+            # Seperate the two valuable parts
+            line_list = line.split(",")
+            for segment in line_list:
+                part = segment.split(":")
+                # Extract key and value
+                values[part[0][-8:]] = int(part[1])
+            return_value[line[:6]] = values
+    return return_value
+
+
+def run_java_acc(src_name, src_jars, dst_name, dst_jars, annotations, skip_annotations):
+    """ Run the compliance checker to compare 'src' and 'dst'. """
+    logging.info("Will check compatibility between original jars:\n\t%s\n"
+                 "and new jars:\n\t%s",
+                 "\n\t".join(src_jars),
+                 "\n\t".join(dst_jars))
+
+    java_acc_path = os.path.join(
+        get_java_acc_dir(), "japi-compliance-checker.pl")
+
+    src_xml_path = os.path.join(get_scratch_dir(), "src.xml")
+    dst_xml_path = os.path.join(get_scratch_dir(), "dst.xml")
+    write_xml_file(src_xml_path, src_name, src_jars)
+    write_xml_file(dst_xml_path, dst_name, dst_jars)
+
+    out_path = os.path.join(get_scratch_dir(), "report.html")
+
+    args = ["perl", java_acc_path,
+            "-l", get_repo_name(),
+            "-d1", src_xml_path,
+            "-d2", dst_xml_path,
+            "-report-path", out_path]
+    if annotations is not None:
+        logging.info("Annotations are: %s", annotations)
+        annotations_path = os.path.join(get_scratch_dir(), "annotations.txt")
+        logging.info("Annotations path: %s", annotations_path)
+        with file(annotations_path, "w") as f:
+            f.write('\n'.join(annotations))
+        args.extend(["-annotations-list", annotations_path])
+
+    if skip_annotations is not None:
+        skip_annotations_path = os.path.join(
+            get_scratch_dir(), "skip_annotations.txt")
+        with file(skip_annotations_path, "w") as f:
+            f.write('\n'.join(skip_annotations))
+        args.extend(["-skip-annotations-list", skip_annotations_path])
+
+    try:
+        output = check_output(args)
+    except subprocess.CalledProcessError as e:
+        # The program returns a nonzero error code if issues are found. We
+        # almost always expect some issues and want to process the results.
+        output = e.output
+    acc_processed = process_java_acc_output(output)
+    return acc_processed
+
+
+def get_known_problems(json_path, src_rev, dst_rev):
+    """ The json file should be in the following format: a dictionary with the
+    keys in the format source_branch/destination_branch and the values
+    dictionaries with binary and source problems and warnings
+    Example:
+    {'branch-1.0.0': {
+      'rel/1.0.0': {'binary': {'problems': 123, 'warnings': 16},
+                      'source': {'problems': 167, 'warnings': 1}},
+      'branch-1.2.0': {'binary': {'problems': 0, 'warnings': 0},
+                      'source': {'problems': 0, 'warnings': 0}}
+      },
+    'branch-1.2.0': {
+      'rel/1.2.1': {'binary': {'problems': 13, 'warnings': 1},
+                      'source': {'problems': 23, 'warnings': 0}}
+      }
+    } """
+    # These are the default values for allowed problems and warnings
+    known_problems = {"binary": {"problems": 0, "warnings": 0},
+                           "source": {"problems": 0, "warnings": 0}}
+    if src_rev.startswith("origin/"):
+      src_rev = src_rev[7:]
+    if dst_rev.startswith("origin/"):
+      dst_rev = dst_rev[7:]
+    if json_path is not None:
+        known_problems = process_json(json_path)
+        try:
+            return known_problems[src_rev][dst_rev]
+        except KeyError:
+            logging.error(("Known Problems values for %s %s are not in "
+                            "provided json file. If you are trying to run "
+                            "the test with the default values, don't "
+                            "provide the --known_problems_path argument")
+                            % (src_rev, dst_rev))
+            raise
+    return known_problems
+
+
+def filter_jars(jars, include_filters, exclude_filters):
+    """ Filter the list of JARs based on include and exclude filters. """
+    filtered = []
+    # Apply include filters
+    for j in jars:
+        basename = os.path.basename(j)
+        for f in include_filters:
+            if f.match(basename):
+                filtered += [j]
+                break
+        else:
+            logging.debug("Ignoring JAR %s", j)
+    # Apply exclude filters
+    exclude_filtered = []
+    for j in filtered:
+        basename = os.path.basename(j)
+        for f in exclude_filters:
+            if f.match(basename):
+                logging.debug("Ignoring JAR %s", j)
+                break
+        else:
+            exclude_filtered += [j]
+
+    return exclude_filtered
+
+
+def main():
+    """ Main function. """
+    logging.basicConfig(level=logging.INFO)
+    parser = argparse.ArgumentParser(
+        description="Run Java API Compliance Checker.")
+    parser.add_argument("-f", "--force-download",
+                        action="store_true",
+                        help="Download dependencies (i.e. Java JAVA_ACC) "
+                        "even if they are already present")
+    parser.add_argument("-i", "--include-file",
+                        action="append",
+                        dest="include_files",
+                        help="Regex filter for JAR files to be included. "
+                        "Applied before the exclude filters. "
+                        "Can be specified multiple times.")
+    parser.add_argument("-e", "--exclude-file",
+                        action="append",
+                        dest="exclude_files",
+                        help="Regex filter for JAR files to be excluded. "
+                        "Applied after the include filters. "
+                        "Can be specified multiple times.")
+    parser.add_argument("-a", "--annotation",
+                        action="append",
+                        dest="annotations",
+                        help="Fully-qualified Java annotation. "
+                        "Java ACC will only check compatibility of "
+                        "annotated classes. Can be specified multiple times.")
+    parser.add_argument("--skip-annotation",
+                        action="append",
+                        dest="skip_annotations",
+                        help="Fully-qualified Java annotation. "
+                        "Java ACC will not check compatibility of "
+                        "these annotated classes. Can be specified multiple "
+                        "times.")
+    parser.add_argument("-p", "--known_problems_path",
+                        default=None, dest="known_problems_path",
+                        help="Path to file with json 'known_problems "
+                        "dictionary.' Path can be relative or absolute. An "
+                        "examples file can be seen in the pydocs for the "
+                        "get_known_problems method.")
+    parser.add_argument("--skip-clean",
+                        action="store_true",
+                        help="Skip cleaning the scratch directory.")
+    parser.add_argument("--compare-warnings", dest="compare_warnings",
+                        action="store_true", default=False,
+                        help="Compare warnings as well as problems.")
+    parser.add_argument("--skip-build",
+                        action="store_true",
+                        help="Skip building the projects.")
+    parser.add_argument("--verbose",
+                        action="store_true",
+                        help="more output")
+    parser.add_argument("src_rev", nargs=1, help="Source revision.")
+    parser.add_argument("dst_rev", nargs="?", default="HEAD",
+                        help="Destination revision. "
+                        "If not specified, will use HEAD.")
+
+    args = parser.parse_args()
+
+    src_rev, dst_rev = args.src_rev[0], args.dst_rev
+
+    logging.info("Source revision: %s", src_rev)
+    logging.info("Destination revision: %s", dst_rev)
+
+    # Configure the expected numbers
+    known_problems = get_known_problems(
+        args.known_problems_path, src_rev, dst_rev)
+
+    # Construct the JAR regex patterns for filtering.
+    include_filters = []
+    if args.include_files is not None:
+        for f in args.include_files:
+            logging.info("Applying JAR filename include filter: %s", f)
+            include_filters += [re.compile(f)]
+    else:
+        include_filters = [re.compile(".*")]
+
+    exclude_filters = []
+    if args.exclude_files is not None:
+        for f in args.exclude_files:
+            logging.info("Applying JAR filename exclude filter: %s", f)
+            exclude_filters += [re.compile(f)]
+
+    # Construct the annotation list
+    if args.annotations is not None:
+        logging.info("Filtering classes using %d annotation(s):",
+                     len(args.annotations))
+        for a in args.annotations:
+            logging.info("\t%s", a)
+
+    skip_annotations = args.skip_annotations
+    if skip_annotations is not None:
+        logging.info("Skipping classes with %d annotation(s):",
+                     len(skip_annotations))
+        for a in skip_annotations:
+            logging.info("\t%s", a)
+
+    # Download deps.
+    checkout_java_acc(args.force_download)
+
+    # Set up the build.
+    scratch_dir = get_scratch_dir()
+    src_dir = os.path.join(scratch_dir, "src")
+    dst_dir = os.path.join(scratch_dir, "dst")
+
+    if args.skip_clean:
+        logging.info("Skipping cleaning the scratch directory")
+    else:
+        clean_scratch_dir(scratch_dir)
+        # Check out the src and dst source trees.
+        checkout_java_tree(get_git_hash(src_rev), src_dir)
+        checkout_java_tree(get_git_hash(dst_rev), dst_dir)
+
+    # Run the build in each.
+    if args.skip_build:
+        logging.info("Skipping the build")
+    else:
+        build_tree(src_dir, args.verbose)
+        build_tree(dst_dir, args.verbose)
+
+    # Find the JARs.
+    src_jars = find_jars(src_dir)
+    dst_jars = find_jars(dst_dir)
+
+    # Filter the JARs.
+    src_jars = filter_jars(src_jars, include_filters, exclude_filters)
+    dst_jars = filter_jars(dst_jars, include_filters, exclude_filters)
+
+    if not src_jars or not dst_jars:
+        logging.error("No JARs found! Are your filters too strong?")
+        sys.exit(1)
+
+    output = run_java_acc(src_rev, src_jars, dst_rev,
+                            dst_jars, args.annotations, skip_annotations)
+    sys.exit(compare_results(output, known_problems,
+                              args.compare_warnings))
+
+
+if __name__ == "__main__":
+    main()

[26/27] hbase git commit: HBASE-18467 WIP build up a jira comment.

Posted by bu...@apache.org.
HBASE-18467 WIP build up a jira comment.


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

Branch: refs/heads/HBASE-18467
Commit: 436d99d5248ac906eb458dd114157847304b80b4
Parents: f314b59
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Aug 9 00:48:46 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Aug 9 00:48:46 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 64 +++++++++++++++++++++++++++++++++++++++++---
 1 file changed, 61 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/436d99d5/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 1f01a47..f87f951 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -128,7 +128,17 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       steps {
         unstash 'yetus'
         // TODO should this be a download from master, similar to how the personality is?
-        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+        sh """
+        declare commentfile
+        if "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+          commentfile='${env.OUTPUTDIR}/success'
+          echo '(/) *{color:green}+1 general checks{color}*' >> "${commentfile}"
+        else
+          commentfile='${env.OUTPUTDIR}/failure'
+          echo '(x) *{color:red}-1 general checks{color}*' >> "${commentfile}"
+        fi
+        echo '-- For more information [see general report|${env.BUILD_URL}/General_Nightly_Build_Report/]' >> "${commentfile}"
+"""
       }
       post {
         always {
@@ -164,7 +174,16 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
           if [ "${env.BRANCH_NAME}" == "branch-1.1" ]; then
             TESTS+=",findbugs"
           fi
-          "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+          declare commentfile
+          if "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+            commentfile='${env.OUTPUTDIR}/success'
+            echo '(/) *{color:green}+1 jdk7 checks{color}*' >> "${commentfile}"
+          else
+            commentfile='${env.OUTPUTDIR}/failure'
+            echo '(x) *{color:red}-1 jdk7 checks{color}*' >> "${commentfile}"
+          fi
+          echo '-- For more information [see jdk7 report|${env.BUILD_URL}/JDK7_Nightly_Build_Report/]' >> "${commentfile}"
+"""
         """
       }
       post {
@@ -215,7 +234,17 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       steps {
         unstash 'yetus'
-        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+        sh """
+          declare commentfile
+          if "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh" ; then
+            commentfile='${env.OUTPUTDIR}/success'
+            echo '(/) *{color:green}+1 jdk8 checks{color}*' >> "${commentfile}"
+          else
+            commentfile='${env.OUTPUTDIR}/failure'
+            echo '(x) *{color:red}-1 jdk8 checks{color}*' >> "${commentfile}"
+          fi
+          echo '-- For more information [see jdk8 report|${env.BUILD_URL}/JDK8_Nightly_Build_Report/]' >> "${commentfile}"
+"""
       }
       post {
         always {
@@ -304,6 +333,35 @@ END
           fi
 '''
       }
+      // This approach only works because the source release artifact is the last stage that does work.
+      post {
+        success {
+          writeFile file: "${env.WORKSPACE}/src_tarball_success", text: '(/) *{color:green}+1 source release artifact{color}*'
+        }
+        failure {
+          writeFile file: "${env.WORKSPACE}/src_tarball_failure", text: '(x) *{color:red}-1 source release artifact{color}*'
+        }
+      }
+    }
+    stage ('Fail if previous stages failed') {
+      steps {
+        script {
+          def failures = ['src_tarball_failure', "${env.OUTPUT_RELATIVE_GENERAL}/failure",
+                          "${env.OUTPUT_RELATIVE_JDK7}/failure", "${OUTPUT_RELATIVE_JDK8}/failure"]
+          for ( failure_file in failures ) {
+            if (fileExists(file: failure_file)) {
+              error 'Failing job due to previous failure(s) in prior steps.'
+            }
+          }
+        }
+      }
+    }
+  }
+  post {
+    always {
+      script {
+         sh "printenv"
+      }
     }
   }
 }

[19/27] hbase git commit: HBASE-18502 Change MasterObserver to use TableDescriptor and ColumnFamilyDescriptor

Posted by bu...@apache.org.
HBASE-18502 Change MasterObserver to use TableDescriptor and ColumnFamilyDescriptor


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

Branch: refs/heads/HBASE-18467
Commit: fd76eb39d727f2794cf9eaa7c81d27dc2bf1036b
Parents: a7014ce
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Mon Aug 7 11:26:15 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Mon Aug 7 11:26:15 2017 +0800

----------------------------------------------------------------------
 .../ExampleMasterObserverWithMetrics.java       |   6 +-
 .../hbase/rsgroup/RSGroupAdminEndpoint.java     |   7 +-
 .../hbase/coprocessor/MasterObserver.java       | 122 ++++++++++---------
 .../hadoop/hbase/coprocessor/package-info.java  |  17 +--
 .../org/apache/hadoop/hbase/master/HMaster.java |  19 +--
 .../hbase/master/MasterCoprocessorHost.java     |  10 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   7 +-
 .../hadoop/hbase/master/MasterServices.java     |   3 +-
 .../security/access/AccessControlLists.java     |   3 +-
 .../hbase/security/access/AccessController.java |  41 +++----
 .../CoprocessorWhitelistMasterObserver.java     |  14 +--
 .../visibility/VisibilityController.java        |   8 +-
 .../backup/TestBackupDeleteWithFailures.java    |   4 +-
 .../hadoop/hbase/client/TestEnableTable.java    |   2 +-
 .../coprocessor/TestCoprocessorMetrics.java     |   5 +-
 ...TestMasterCoprocessorExceptionWithAbort.java |   3 +-
 ...estMasterCoprocessorExceptionWithRemove.java |   3 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  54 ++++----
 .../hbase/master/MockNoopMasterServices.java    |   3 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |   5 +-
 .../hbase/security/access/SecureTestUtil.java   |   3 +-
 .../access/TestWithDisabledAuthorization.java   |   6 +-
 .../snapshot/TestSnapshotClientRetries.java     |   6 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |   4 +-
 24 files changed, 184 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
index 6649162..8535d05 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
@@ -24,8 +24,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -68,7 +68,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                             HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                             TableDescriptor desc, HRegionInfo[] regions) throws IOException {
     // we rely on the fact that there is only 1 instance of our MasterObserver. We keep track of
     // when the operation starts before the operation is executing.
     this.createTableStartTime = System.currentTimeMillis();
@@ -76,7 +76,7 @@ public class ExampleMasterObserverWithMetrics implements MasterObserver {
 
   @Override
   public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                              HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                              TableDescriptor desc, HRegionInfo[] regions) throws IOException {
     if (this.createTableStartTime > 0) {
       long time = System.currentTimeMillis() - this.createTableStartTime;
       LOG.info("Create table took: " + time);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index 91d31d0..9fda3f0 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -268,7 +269,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
     }
   }
 
-  void assignTableToGroup(HTableDescriptor desc) throws IOException {
+  void assignTableToGroup(TableDescriptor desc) throws IOException {
     String groupName =
         master.getClusterSchema().getNamespace(desc.getTableName().getNamespaceAsString())
                 .getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP);
@@ -293,7 +294,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
   // Assign table to default RSGroup.
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+      TableDescriptor desc, HRegionInfo[] regions) throws IOException {
     assignTableToGroup(desc);
   }
 
@@ -330,7 +331,7 @@ public class RSGroupAdminEndpoint implements MasterObserver, CoprocessorService
 
   @Override
   public void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      SnapshotDescription snapshot, HTableDescriptor desc) throws IOException {
+      SnapshotDescription snapshot, TableDescriptor desc) throws IOException {
     assignTableToGroup(desc);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index cf75c72..f4f5db3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -35,8 +35,10 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -83,21 +85,21 @@ public interface MasterObserver extends Coprocessor {
    * table RPC call.
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
    */
   default void preCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
+      TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
 
   /**
    * Called after the createTable operation has been requested.  Called as part
    * of create table RPC call.
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
    */
   default void postCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
+      TableDescriptor desc, HRegionInfo[] regions) throws IOException {}
 
   /**
    * Called before a new table is created by
@@ -109,7 +111,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regions the initial regions created for the table
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])}.
+   *     Use {@link #preCreateTableAction(ObserverContext, TableDescriptor, HRegionInfo[])}.
    */
   @Deprecated
   default void preCreateTableHandler(final ObserverContext<MasterCoprocessorEnvironment>
@@ -124,7 +126,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regions the initial regions created for the table
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *   (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *   Use {@link #postCompletedCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])}
+   *   Use {@link #postCompletedCreateTableAction(ObserverContext, TableDescriptor, HRegionInfo[])}
    */
   @Deprecated
   default void postCreateTableHandler(final ObserverContext<MasterCoprocessorEnvironment>
@@ -141,12 +143,12 @@ public interface MasterObserver extends Coprocessor {
    * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
    */
   default void preCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HTableDescriptor desc,
+      final TableDescriptor desc,
       final HRegionInfo[] regions) throws IOException {}
 
   /**
@@ -159,12 +161,12 @@ public interface MasterObserver extends Coprocessor {
    * Make sure to implement only one of the two as both are called.
    *
    * @param ctx the environment to interact with the framework and master
-   * @param desc the HTableDescriptor for the table
+   * @param desc the TableDescriptor for the table
    * @param regions the initial regions created for the table
    */
   default void postCompletedCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final HTableDescriptor desc,
+      final TableDescriptor desc,
       final HRegionInfo[] regions) throws IOException {}
 
   /**
@@ -345,20 +347,20 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the TableDescriptor
    */
   default void preModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, HTableDescriptor htd) throws IOException {}
+      final TableName tableName, TableDescriptor htd) throws IOException {}
 
   /**
    * Called after the modifyTable operation has been requested.  Called as part
    * of modify table RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the TableDescriptor
    */
   default void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final TableName tableName, HTableDescriptor htd) throws IOException {}
+      final TableName tableName, TableDescriptor htd) throws IOException {}
 
   /**
    * Called prior to modifying a table's properties.  Called as part of modify
@@ -369,7 +371,7 @@ public interface MasterObserver extends Coprocessor {
    * @param htd the HTableDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-15575">HBASE-15575</a>).
-   *     Use {@link #preModifyTableAction(ObserverContext, TableName, HTableDescriptor)}.
+   *     Use {@link #preModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
    */
   @Deprecated
   default void preModifyTableHandler(
@@ -385,7 +387,7 @@ public interface MasterObserver extends Coprocessor {
    * @param htd the HTableDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *     Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, HTableDescriptor)}.
+   *     Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, TableDescriptor)}.
    */
   @Deprecated
   default void postModifyTableHandler(
@@ -403,12 +405,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the TableDescriptor
    */
   default void preModifyTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HTableDescriptor htd) throws IOException {}
+      final TableDescriptor htd) throws IOException {}
 
   /**
    * Called after to modifying a table's properties.  Called as part of modify
@@ -421,12 +423,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param htd the HTableDescriptor
+   * @param htd the TableDescriptor
    */
   default void postCompletedModifyTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HTableDescriptor htd) throws IOException {}
+      final TableDescriptor htd) throws IOException {}
 
   /**
    * Called prior to adding a new column family to the table.  Called as part of
@@ -436,7 +438,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}.
+   *             Use {@link #preAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void preAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -452,10 +454,10 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void preAddColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
+      TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called after the new column family has been created.  Called as part of
@@ -465,7 +467,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #postAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}.
+   *             Use {@link #postAddColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void postAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -481,10 +483,10 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void postAddColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
+      TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called prior to adding a new column family to the table.  Called as part of
@@ -494,7 +496,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *          (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *          {@link #preAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}.
+   *          {@link #preAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void preAddColumnHandler(
@@ -511,12 +513,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void preAddColumnFamilyAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HColumnDescriptor columnFamily) throws IOException {}
+      final ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called after the new column family has been created.  Called as part of
@@ -526,7 +528,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *     {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}.
+   *     {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void postAddColumnHandler(
@@ -543,12 +545,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void postCompletedAddColumnFamilyAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HColumnDescriptor columnFamily) throws IOException {}
+      final ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
@@ -558,7 +560,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #preModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}.
+   *             Use {@link #preModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void preModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -574,10 +576,10 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void preModifyColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
+      TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called after the column family has been updated.  Called as part of modify
@@ -587,7 +589,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *             Use {@link #postModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}.
+   *             Use {@link #postModifyColumnFamily(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void postModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
@@ -603,10 +605,10 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void postModifyColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {}
+      TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
@@ -616,7 +618,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *     (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>).
-   *     Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}.
+   *     Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void preModifyColumnHandler(
@@ -633,12 +635,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void preModifyColumnFamilyAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HColumnDescriptor columnFamily) throws IOException {}
+      final ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called after the column family has been updated.  Called as part of modify
@@ -648,7 +650,7 @@ public interface MasterObserver extends Coprocessor {
    * @param columnFamily the HColumnDescriptor
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *   (<a href="https://issues.apache.org/jira/browse/HBASE-13645">HBASE-13645</a>). Use
-   *   {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,HColumnDescriptor)}.
+   *   {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,ColumnFamilyDescriptor)}.
    */
   @Deprecated
   default void postModifyColumnHandler(
@@ -665,12 +667,12 @@ public interface MasterObserver extends Coprocessor {
    *
    * @param ctx the environment to interact with the framework and master
    * @param tableName the name of the table
-   * @param columnFamily the HColumnDescriptor
+   * @param columnFamily the ColumnFamilyDescriptor
    */
   default void postCompletedModifyColumnFamilyAction(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
       final TableName tableName,
-      final HColumnDescriptor columnFamily) throws IOException {}
+      final ColumnFamilyDescriptor columnFamily) throws IOException {}
 
   /**
    * Called prior to deleting the entire column family.  Called as part of
@@ -1282,10 +1284,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to snapshot
+   * @param hTableDescriptor the TableDescriptor of the table to snapshot
    */
   default void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1293,10 +1295,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of snapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to snapshot
+   * @param hTableDescriptor the TableDescriptor of the table to snapshot
    */
   default void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1323,10 +1325,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to create
+   * @param hTableDescriptor the TableDescriptor of the table to create
    */
   default void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1334,10 +1336,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of restoreSnapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to create
+   * @param hTableDescriptor the v of the table to create
    */
   default void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1346,10 +1348,10 @@ public interface MasterObserver extends Coprocessor {
    * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to restore
+   * @param hTableDescriptor the TableDescriptor of the table to restore
    */
   default void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1357,10 +1359,10 @@ public interface MasterObserver extends Coprocessor {
    * Called as part of restoreSnapshot RPC call.
    * @param ctx the environment to interact with the framework and master
    * @param snapshot the SnapshotDescriptor for the snapshot
-   * @param hTableDescriptor the hTableDescriptor of the table to restore
+   * @param hTableDescriptor the TableDescriptor of the table to restore
    */
   default void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {}
 
   /**
@@ -1390,7 +1392,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regex regular expression used for filtering the table names
    */
   default void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+      List<TableName> tableNamesList, List<TableDescriptor> descriptors,
       String regex) throws IOException {}
 
   /**
@@ -1401,7 +1403,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regex regular expression used for filtering the table names
    */
   default void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+      List<TableName> tableNamesList, List<TableDescriptor> descriptors,
       String regex) throws IOException {}
 
   /**
@@ -1411,7 +1413,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regex regular expression used for filtering the table names
    */
   default void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<HTableDescriptor> descriptors, String regex) throws IOException {}
+      List<TableDescriptor> descriptors, String regex) throws IOException {}
 
   /**
    * Called after a getTableNames request has been processed.
@@ -1420,7 +1422,7 @@ public interface MasterObserver extends Coprocessor {
    * @param regex regular expression used for filtering the table names
    */
   default void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<HTableDescriptor> descriptors, String regex) throws IOException {}
+      List<TableDescriptor> descriptors, String regex) throws IOException {}
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java
index e2113c0..8a677ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java
@@ -192,7 +192,7 @@ code, see the {@link org.apache.hadoop.hbase.client.coprocessor} package documen
 
 <h2><a name="load">Coprocessor loading</a></h2>
 A customized coprocessor can be loaded by two different ways, by configuration,
-or by <code>HTableDescriptor</code> for a newly created table.
+or by <code>TableDescriptor</code> for a newly created table.
 <p>
 (Currently we don't really have an on demand coprocessor loading mechanism for
 opened regions.)
@@ -255,13 +255,14 @@ policy implementations, perhaps) ahead of observers.
     "TestClassloading.jar");
 
   // create a table that references the jar
-  HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getClass().getTableName()));
-  htd.addFamily(new HColumnDescriptor("test"));
-  htd.setValue("Coprocessor$1",
-    path.toString() +
-    ":" + classFullName +
-    ":" + Coprocessor.Priority.USER);
-  HBaseAdmin admin = new HBaseAdmin(this.conf);
+  TableDescriptor htd = TableDescriptorBuilder
+                        .newBuilder(TableName.valueOf(getClass().getTableName()))
+                        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("test"))
+                        .setValue(Bytes.toBytes("Coprocessor$1", path.toString()+
+                          ":" + classFullName +
+                          ":" + Coprocessor.Priority.USER))
+                        .build();
+  Admin admin = connection.getAdmin();
   admin.createTable(htd);
 </pre></blockquote>
 <h3>Chain of RegionObservers</h3>

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index d6b149a..96bf859 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -53,6 +53,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -2975,7 +2976,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
     checkInitialized();
     return listTableDescriptors(name, null, null, true);
   }
@@ -3042,10 +3043,10 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @param includeSysTables False to match only against userspace tables
    * @return the list of table descriptors
    */
-  public List<HTableDescriptor> listTableDescriptors(final String namespace, final String regex,
+  public List<TableDescriptor> listTableDescriptors(final String namespace, final String regex,
       final List<TableName> tableNameList, final boolean includeSysTables)
   throws IOException {
-    List<HTableDescriptor> htds = new ArrayList<>();
+    List<TableDescriptor> htds = new ArrayList<>();
     boolean bypass = cpHost != null?
         cpHost.preGetTableDescriptors(tableNameList, htds, regex): false;
     if (!bypass) {
@@ -3066,14 +3067,14 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   public List<TableName> listTableNames(final String namespace, final String regex,
       final boolean includeSysTables) throws IOException {
-    List<HTableDescriptor> htds = new ArrayList<>();
+    List<TableDescriptor> htds = new ArrayList<>();
     boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
     if (!bypass) {
       htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
       if (cpHost != null) cpHost.postGetTableNames(htds, regex);
     }
     List<TableName> result = new ArrayList<>(htds.size());
-    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
+    for (TableDescriptor htd: htds) result.add(htd.getTableName());
     return result;
   }
 
@@ -3082,7 +3083,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    *    tables, etc.
    * @throws IOException
    */
-  private List<HTableDescriptor> getTableDescriptors(final List<HTableDescriptor> htds,
+  private List<TableDescriptor> getTableDescriptors(final List<TableDescriptor> htds,
       final String namespace, final String regex, final List<TableName> tableNameList,
       final boolean includeSysTables)
   throws IOException {
@@ -3123,12 +3124,12 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @param descriptors list of table descriptors to filter
    * @param pattern the regex to use
    */
-  private static void filterTablesByRegex(final Collection<HTableDescriptor> descriptors,
+  private static void filterTablesByRegex(final Collection<TableDescriptor> descriptors,
       final Pattern pattern) {
     final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
-    Iterator<HTableDescriptor> itr = descriptors.iterator();
+    Iterator<TableDescriptor> itr = descriptors.iterator();
     while (itr.hasNext()) {
-      HTableDescriptor htd = itr.next();
+      TableDescriptor htd = itr.next();
       String tableName = htd.getTableName().getNameAsString();
       boolean matched = pattern.matcher(tableName).matches();
       if (!matched && htd.getTableName().getNamespaceAsString().equals(defaultNS)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 6064f9b..04bdacf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.commons.lang.ClassUtils;
 import org.apache.commons.logging.Log;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -1259,7 +1261,7 @@ public class MasterCoprocessorHost
   }
 
   public boolean preGetTableDescriptors(final List<TableName> tableNamesList,
-      final List<HTableDescriptor> descriptors, final String regex) throws IOException {
+      final List<TableDescriptor> descriptors, final String regex) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1270,7 +1272,7 @@ public class MasterCoprocessorHost
   }
 
   public void postGetTableDescriptors(final List<TableName> tableNamesList,
-      final List<HTableDescriptor> descriptors, final String regex) throws IOException {
+      final List<TableDescriptor> descriptors, final String regex) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
       public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@@ -1280,7 +1282,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public boolean preGetTableNames(final List<HTableDescriptor> descriptors,
+  public boolean preGetTableNames(final List<TableDescriptor> descriptors,
       final String regex) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override
@@ -1291,7 +1293,7 @@ public class MasterCoprocessorHost
     });
   }
 
-  public void postGetTableNames(final List<HTableDescriptor> descriptors,
+  public void postGetTableNames(final List<TableDescriptor> descriptors,
       final String regex) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index aa64caa..5a2cd17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
@@ -856,13 +857,13 @@ public class MasterRpcServices extends RSRpcServices
         }
       }
 
-      List<HTableDescriptor> descriptors = master.listTableDescriptors(namespace, regex,
+      List<TableDescriptor> descriptors = master.listTableDescriptors(namespace, regex,
           tableNameList, req.getIncludeSysTables());
 
       GetTableDescriptorsResponse.Builder builder = GetTableDescriptorsResponse.newBuilder();
       if (descriptors != null && descriptors.size() > 0) {
         // Add the table descriptors to the response
-        for (HTableDescriptor htd: descriptors) {
+        for (TableDescriptor htd: descriptors) {
           builder.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
         }
       }
@@ -1114,7 +1115,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListTableDescriptorsByNamespaceResponse.Builder b =
           ListTableDescriptorsByNamespaceResponse.newBuilder();
-      for (HTableDescriptor htd : master
+      for (TableDescriptor htd : master
           .listTableDescriptorsByNamespace(request.getNamespaceName())) {
         b.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 3046b8a..f7f5d06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
@@ -381,7 +382,7 @@ public interface MasterServices extends Server {
    * @return descriptors
    * @throws IOException
    */
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException;
+  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException;
 
   /**
    * Get list of table names by namespace

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 97ac1de..dfadbb9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
@@ -365,7 +366,7 @@ public class AccessControlLists {
   /**
    * Returns {@code true} if the given table is {@code _acl_} metadata table.
    */
-  static boolean isAclTable(HTableDescriptor desc) {
+  static boolean isAclTable(TableDescriptor desc) {
     return ACL_TABLE_NAME.equals(desc.getTableName());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 42de48d..c40d481 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -43,10 +43,8 @@ import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -57,6 +55,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
@@ -68,6 +67,7 @@ import org.apache.hadoop.hbase.client.Query;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -134,7 +134,6 @@ import com.google.protobuf.Message;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 
 /**
  * Provides basic authorization checks for data access and administrative
@@ -988,8 +987,8 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
-    Set<byte[]> families = desc.getFamiliesKeys();
+      TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+    Set<byte[]> families = desc.getColumnFamilyNames();
     Map<byte[], Set<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (byte[] family: families) {
       familyMap.put(family, null);
@@ -1001,7 +1000,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
   @Override
   public void postCompletedCreateTableAction(
       final ObserverContext<MasterCoprocessorEnvironment> c,
-      final HTableDescriptor desc,
+      final TableDescriptor desc,
       final HRegionInfo[] regions) throws IOException {
     // When AC is used, it should be configured as the 1st CP.
     // In Master, the table operations like create, are handled by a Thread pool but the max size
@@ -1108,14 +1107,14 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
-      HTableDescriptor htd) throws IOException {
+      TableDescriptor htd) throws IOException {
     requirePermission(getActiveUser(c), "modifyTable", tableName, null, null,
         Action.ADMIN, Action.CREATE);
   }
 
   @Override
   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
-      TableName tableName, final HTableDescriptor htd) throws IOException {
+      TableName tableName, final TableDescriptor htd) throws IOException {
     final Configuration conf = c.getEnvironment().getConfiguration();
     // default the table owner to current user, if not specified.
     final String owner = (htd.getOwnerString() != null) ? htd.getOwnerString() :
@@ -1134,7 +1133,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                                 TableName tableName, HColumnDescriptor columnFamily)
+                                 TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
     requireTablePermission(getActiveUser(ctx), "addColumn", tableName, columnFamily.getName(), null,
         Action.ADMIN, Action.CREATE);
@@ -1142,7 +1141,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                                    TableName tableName, HColumnDescriptor columnFamily)
+                                    TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
     requirePermission(getActiveUser(ctx), "modifyColumn", tableName, columnFamily.getName(), null,
         Action.ADMIN, Action.CREATE);
@@ -1318,7 +1317,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {
     requirePermission(getActiveUser(ctx), "snapshot " + snapshot.getName(), hTableDescriptor.getTableName(), null, null,
       Permission.Action.ADMIN);
@@ -1340,11 +1339,11 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {
     User user = getActiveUser(ctx);
     if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)
-        && hTableDescriptor.getNameAsString().equals(snapshot.getTable())) {
+        && hTableDescriptor.getTableName().getNameAsString().equals(snapshot.getTable())) {
       // Snapshot owner is allowed to create a table with the same name as the snapshot he took
       AuthResult result = AuthResult.allow("cloneSnapshot " + snapshot.getName(),
         "Snapshot owner check allowed", user, null, hTableDescriptor.getTableName(), null);
@@ -1356,7 +1355,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+      final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
       throws IOException {
     User user = getActiveUser(ctx);
     if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, user)) {
@@ -2521,7 +2520,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-       List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+       List<TableName> tableNamesList, List<TableDescriptor> descriptors,
        String regex) throws IOException {
     // We are delegating the authorization check to postGetTableDescriptors as we don't have
     // any concrete set of table names when a regex is present or the full list is requested.
@@ -2541,7 +2540,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+      List<TableName> tableNamesList, List<TableDescriptor> descriptors,
       String regex) throws IOException {
     // Skipping as checks in this case are already done by preGetTableDescriptors.
     if (regex == null && tableNamesList != null && !tableNamesList.isEmpty()) {
@@ -2550,9 +2549,9 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
     // Retains only those which passes authorization checks, as the checks weren't done as part
     // of preGetTableDescriptors.
-    Iterator<HTableDescriptor> itr = descriptors.iterator();
+    Iterator<TableDescriptor> itr = descriptors.iterator();
     while (itr.hasNext()) {
-      HTableDescriptor htd = itr.next();
+      TableDescriptor htd = itr.next();
       try {
         requirePermission(getActiveUser(ctx), "getTableDescriptors", htd.getTableName(), null, null,
             Action.ADMIN, Action.CREATE);
@@ -2564,11 +2563,11 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
 
   @Override
   public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<HTableDescriptor> descriptors, String regex) throws IOException {
+      List<TableDescriptor> descriptors, String regex) throws IOException {
     // Retains only those which passes authorization checks.
-    Iterator<HTableDescriptor> itr = descriptors.iterator();
+    Iterator<TableDescriptor> itr = descriptors.iterator();
     while (itr.hasNext()) {
-      HTableDescriptor htd = itr.next();
+      TableDescriptor htd = itr.next();
       try {
         requireAccess(getActiveUser(ctx), "getTableNames", htd.getTableName(), Action.values());
       } catch (AccessDeniedException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
index 5771593..9a94e89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.java
@@ -19,10 +19,7 @@
 package org.apache.hadoop.hbase.security.access;
 
 import java.io.IOException;
-import java.net.URI;
-import java.nio.file.PathMatcher;
 import java.util.Collection;
-import java.util.List;
 import java.util.regex.Matcher;
 
 import org.apache.commons.io.FilenameUtils;
@@ -39,9 +36,9 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -58,13 +55,13 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
 
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HTableDescriptor htd) throws IOException {
+      TableName tableName, TableDescriptor htd) throws IOException {
     verifyCoprocessors(ctx, htd);
   }
 
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor htd, HRegionInfo[] regions) throws IOException {
+      TableDescriptor htd, HRegionInfo[] regions) throws IOException {
     verifyCoprocessors(ctx, htd);
   }
 
@@ -143,7 +140,7 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
    * @param  htd         as passed in from the coprocessor
    */
   private void verifyCoprocessors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      HTableDescriptor htd) throws IOException {
+      TableDescriptor htd) throws IOException {
 
     MasterServices services = ctx.getEnvironment().getMasterServices();
     Configuration conf = services.getConfiguration();
@@ -152,9 +149,8 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
         conf.getStringCollection(
             CP_COPROCESSOR_WHITELIST_PATHS_KEY);
 
-    List<String> coprocs = htd.getCoprocessors();
+    Collection<String> coprocs = htd.getCoprocessors();
     for (int i = 0; i < coprocs.size(); i++) {
-      String coproc = coprocs.get(i);
 
       String coprocSpec = Bytes.toString(htd.getValue(
           Bytes.toBytes("coprocessor$" + (i + 1))));

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index c6091c3..23f0583 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
@@ -59,6 +60,7 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -215,7 +217,7 @@ public class VisibilityController implements MasterObserver, RegionObserver,
 
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HTableDescriptor htd) throws IOException {
+      TableName tableName, TableDescriptor htd) throws IOException {
     if (!authorizationEnabled) {
       return;
     }
@@ -226,7 +228,7 @@ public class VisibilityController implements MasterObserver, RegionObserver,
 
   @Override
   public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                                 TableName tableName, HColumnDescriptor columnFamily)
+                                 TableName tableName, ColumnFamilyDescriptor columnFamily)
       throws IOException {
     if (!authorizationEnabled) {
       return;
@@ -238,7 +240,7 @@ public class VisibilityController implements MasterObserver, RegionObserver,
 
   @Override
   public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      TableName tableName, HColumnDescriptor columnFamily) throws IOException {
+      TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
     if (!authorizationEnabled) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
index 59309ee..966f519 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
@@ -30,11 +30,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
@@ -81,7 +81,7 @@ public class TestBackupDeleteWithFailures extends TestBackupBase{
 
     @Override
     public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException
     {
        if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
index a0cf4d2..133b111 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
@@ -197,7 +197,7 @@ public class TestEnableTable {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HTableDescriptor desc,
+        final TableDescriptor desc,
         final HRegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index f997751..878d445 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.metrics.Counter;
 import org.apache.hadoop.hbase.metrics.Metric;
@@ -101,14 +102,14 @@ public class TestCoprocessorMetrics {
 
     @Override
     public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                               HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                               TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       // we rely on the fact that there is only 1 instance of our MasterObserver
       this.start = System.currentTimeMillis();
     }
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-                                HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+                                TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       if (this.start > 0) {
         long time = System.currentTimeMillis() - start;
         LOG.info("Create table took: " + time);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
index 5130a41..c0a5801 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -104,7 +105,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       // cause a NullPointerException and don't catch it: this will cause the
       // master to abort().
       Integer i;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
index 7c5d8a1..92d12ee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -81,7 +82,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
     @SuppressWarnings("null")
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       // Cause a NullPointerException and don't catch it: this should cause the
       // master to throw an o.apache.hadoop.hbase.DoNotRetryIOException to the
       // client.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 1e6d717..1b8b27b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -46,12 +46,14 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.RegionPlan;
@@ -316,7 +318,7 @@ public class TestMasterObserver {
 
     @Override
     public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -325,7 +327,7 @@ public class TestMasterObserver {
 
     @Override
     public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       postCreateTableCalled = true;
     }
 
@@ -396,7 +398,7 @@ public class TestMasterObserver {
 
     @Override
     public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableName tableName, HTableDescriptor htd) throws IOException {
+        TableName tableName, TableDescriptor htd) throws IOException {
       if (bypass) {
         env.bypass();
       }else{
@@ -407,7 +409,7 @@ public class TestMasterObserver {
 
     @Override
     public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
-        TableName tableName, HTableDescriptor htd) throws IOException {
+        TableName tableName, TableDescriptor htd) throws IOException {
       postModifyTableCalled = true;
     }
 
@@ -537,7 +539,7 @@ public class TestMasterObserver {
 
     @Override
     public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily
+        TableName tableName, ColumnFamilyDescriptor columnFamily
     ) throws IOException {
       if (bypass) {
         ctx.bypass();
@@ -556,7 +558,7 @@ public class TestMasterObserver {
 
     @Override
     public void postAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
+        TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
       postAddColumnCalled = true;
     }
 
@@ -576,7 +578,7 @@ public class TestMasterObserver {
 
     @Override
     public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
+        TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
       if (bypass) {
         ctx.bypass();
       }
@@ -591,7 +593,7 @@ public class TestMasterObserver {
 
     @Override
     public void postModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableName tableName, HColumnDescriptor columnFamily) throws IOException {
+        TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
       postModifyColumnCalled = true;
     }
 
@@ -941,14 +943,14 @@ public class TestMasterObserver {
 
     @Override
     public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       preSnapshotCalled = true;
     }
 
     @Override
     public void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       postSnapshotCalled = true;
     }
@@ -975,14 +977,14 @@ public class TestMasterObserver {
 
     @Override
     public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       preCloneSnapshotCalled = true;
     }
 
     @Override
     public void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       postCloneSnapshotCalled = true;
     }
@@ -993,14 +995,14 @@ public class TestMasterObserver {
 
     @Override
     public void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       preRestoreSnapshotCalled = true;
     }
 
     @Override
     public void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       postRestoreSnapshotCalled = true;
     }
@@ -1035,7 +1037,7 @@ public class TestMasterObserver {
     @Override
     public void preCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
-        final HTableDescriptor desc,
+        final TableDescriptor desc,
         final HRegionInfo[] regions) throws IOException {
       if (bypass) {
         env.bypass();
@@ -1053,7 +1055,7 @@ public class TestMasterObserver {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HTableDescriptor desc,
+        final TableDescriptor desc,
         final HRegionInfo[] regions) throws IOException {
       postCompletedCreateTableActionCalled = true;
       tableCreationLatch.countDown();
@@ -1167,7 +1169,7 @@ public class TestMasterObserver {
     public void preModifyTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
         final TableName tableName,
-        final HTableDescriptor htd) throws IOException {
+        final TableDescriptor htd) throws IOException {
       if (bypass) {
         env.bypass();
       }
@@ -1178,7 +1180,7 @@ public class TestMasterObserver {
     public void postCompletedModifyTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> env,
         final TableName tableName,
-        final HTableDescriptor htd) throws IOException {
+        final TableDescriptor htd) throws IOException {
       postCompletedModifyTableActionCalled = true;
     }
 
@@ -1201,7 +1203,7 @@ public class TestMasterObserver {
     public void preAddColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableName tableName,
-        final HColumnDescriptor columnFamily) throws IOException {
+        final ColumnFamilyDescriptor columnFamily) throws IOException {
       if (bypass) {
         ctx.bypass();
       }
@@ -1219,7 +1221,7 @@ public class TestMasterObserver {
     public void postCompletedAddColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableName tableName,
-        final HColumnDescriptor columnFamily) throws IOException {
+        final ColumnFamilyDescriptor columnFamily) throws IOException {
       postCompletedAddColumnFamilyActionCalled = true;
     }
 
@@ -1242,7 +1244,7 @@ public class TestMasterObserver {
     public void preModifyColumnFamilyAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableName tableName,
-        final HColumnDescriptor columnFamily) throws IOException {
+        final ColumnFamilyDescriptor columnFamily) throws IOException {
       if (bypass) {
         ctx.bypass();
       }
@@ -1259,7 +1261,7 @@ public class TestMasterObserver {
     @Override
     public void postCompletedModifyColumnFamilyAction(
         ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
-        HColumnDescriptor columnFamily) throws IOException {
+        ColumnFamilyDescriptor columnFamily) throws IOException {
       postCompletedModifyColumnFamilyActionCalled = true;
     }
 
@@ -1390,14 +1392,14 @@ public class TestMasterObserver {
 
     @Override
     public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        List<TableName> tableNamesList, List<HTableDescriptor> descriptors, String regex)
+        List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex)
         throws IOException {
       preGetTableDescriptorsCalled = true;
     }
 
     @Override
     public void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        List<TableName> tableNamesList, List<HTableDescriptor> descriptors,
+        List<TableName> tableNamesList, List<TableDescriptor> descriptors,
         String regex) throws IOException {
       postGetTableDescriptorsCalled = true;
     }
@@ -1408,13 +1410,13 @@ public class TestMasterObserver {
 
     @Override
     public void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        List<HTableDescriptor> descriptors, String regex) throws IOException {
+        List<TableDescriptor> descriptors, String regex) throws IOException {
       preGetTableNamesCalled = true;
     }
 
     @Override
     public void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        List<HTableDescriptor> descriptors, String regex) throws IOException {
+        List<TableDescriptor> descriptors, String regex) throws IOException {
       postGetTableNamesCalled = true;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 3c4dc94..1636ba5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -236,7 +237,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
     return null;  //To change body of implemented methods use File | Settings | File Templates.
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 37537e5..f641887 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -25,8 +25,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
@@ -58,6 +56,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
@@ -554,7 +553,7 @@ public class TestNamespaceAuditor {
 
     @Override
     public void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       if (throwExceptionInPreCreateTableAction) {
         throw new IOException("Throw exception as it is demanded.");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index e47cfd5..5627016 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -629,7 +630,7 @@ public class SecureTestUtil {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
index afc5844..bfd9c1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -44,12 +44,12 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -704,7 +704,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
       public Object run() throws Exception {
         List<TableName> tableNamesList = Lists.newArrayList();
         tableNamesList.add(TEST_TABLE.getTableName());
-        List<HTableDescriptor> descriptors = Lists.newArrayList();
+        List<TableDescriptor> descriptors = Lists.newArrayList();
         ACCESS_CONTROLLER.preGetTableDescriptors(ObserverContext.createAndPrepare(CP_ENV, null),
           tableNamesList, descriptors, ".+");
         return null;
@@ -715,7 +715,7 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
     verifyAllowed(new AccessTestAction() {
       @Override
       public Object run() throws Exception {
-        List<HTableDescriptor> descriptors = Lists.newArrayList();
+        List<TableDescriptor> descriptors = Lists.newArrayList();
         ACCESS_CONTROLLER.preGetTableNames(ObserverContext.createAndPrepare(CP_ENV, null),
           descriptors, ".+");
         return null;

[10/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
----------------------------------------------------------------------
diff --git a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom b/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
deleted file mode 100644
index d12092b..0000000
--- a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
+++ /dev/null
@@ -1,718 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-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.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <groupId>org.apache.maven.skins</groupId>
-    <artifactId>maven-skins</artifactId>
-    <version>10</version>
-    <relativePath>../maven-skins/pom.xml</relativePath>
-  </parent>
-
-  <artifactId>maven-fluido-skin</artifactId>
-  <version>1.5-HBASE</version>
-
-  <name>Apache Maven Fluido Skin</name>
-  <description>The Apache Maven Fluido Skin is an Apache Maven site skin
-    built on top of Twitter's bootstrap.</description>
-  <inceptionYear>2011</inceptionYear>
-
-  <scm>
-    <connection>scm:svn:http://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</connection>
-    <developerConnection>scm:svn:https://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</developerConnection>
-    <url>http://svn.apache.org/viewvc/maven/skins/trunk/maven-fluido-skin/</url>
-  </scm>
-  <issueManagement>
-    <system>jira</system>
-    <url>https://issues.apache.org/jira/browse/MSKINS/component/12326474</url>
-  </issueManagement>
-  <distributionManagement>
-    <site>
-      <id>apache.website</id>
-      <url>scm:svn:https://svn.apache.org/repos/infra/websites/production/maven/components/${maven.site.path}</url>
-    </site>
-  </distributionManagement>
-
-  <contributors>
-    <!-- in alphabetical order -->
-    <contributor>
-      <name>Bruno P. Kinoshita</name>
-      <email>brunodepaulak AT yahoo DOT com DOT br</email>
-    </contributor>
-    <contributor>
-      <name>Carlos Villaronga</name>
-      <email>cvillaronga AT gmail DOT com</email>
-    </contributor>
-    <contributor>
-      <name>Christian Grobmeier</name>
-      <email>grobmeier AT apache DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Conny Kreyssel</name>
-      <email>dev AT kreyssel DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Michael Koch</name>
-      <email>tensberg AT gmx DOT net</email>
-    </contributor>
-    <contributor>
-      <name>Emmanuel Hugonnet</name>
-      <email>emmanuel DOT hugonnet AT gmail DOT com</email>
-    </contributor>
-    <contributor>
-      <name>Ivan Habunek</name>
-      <email>ihabunek AT apache DOT org</email>
-    </contributor>
-    <contributor>
-      <name>Eric Barboni</name>
-    </contributor>
-    <contributor>
-      <name>Michael Osipov</name>
-      <email>michaelo AT apache DOT org</email>
-    </contributor>
-  </contributors>
-
-  <properties>
-    <bootstrap.version>2.3.2</bootstrap.version>
-    <jquery.version>1.11.2</jquery.version>
-  </properties>
-
-  <build>
-    <resources>
-      <resource>
-        <directory>.</directory>
-        <targetPath>META-INF</targetPath>
-        <includes>
-          <include>NOTICE</include>
-          <include>LICENSE</include>
-        </includes>
-      </resource>
-
-      <!-- exclude css and js since will include the minified version -->
-      <resource>
-        <directory>${basedir}/src/main/resources</directory>
-        <excludes>
-          <exclude>css/**</exclude>
-          <exclude>js/**</exclude>
-        </excludes>
-        <filtering>true</filtering> <!-- add skin-info -->
-      </resource>
-
-      <!-- include the print.css -->
-      <resource>
-        <directory>${basedir}/src/main/resources</directory>
-        <includes>
-          <include>css/print.css</include>
-        </includes>
-      </resource>
-
-      <!-- include minified only -->
-      <resource>
-        <directory>${project.build.directory}/${project.build.finalName}</directory>
-        <includes>
-          <include>css/apache-maven-fluido-${project.version}.min.css</include>
-          <include>js/apache-maven-fluido-${project.version}.min.js</include>
-        </includes>
-      </resource>
-    </resources>
-
-    <pluginManagement>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.rat</groupId>
-          <artifactId>apache-rat-plugin</artifactId>
-          <configuration>
-            <excludes combine.children="append">
-              <exclude>src/main/resources/fonts/glyphicons-halflings-regular.svg</exclude>
-              <exclude>src/main/resources/js/prettify.js</exclude>
-              <exclude>src/main/resources/js/jquery-*.js</exclude>
-            </excludes>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-resources-plugin</artifactId>
-        <dependencies><!-- TODO remove when upgrading to version 2.8: see MSHARED-325 / MRESOURCES-192 -->
-          <dependency>
-              <groupId>org.apache.maven.shared</groupId>
-              <artifactId>maven-filtering</artifactId>
-              <version>1.3</version>
-          </dependency>
-        </dependencies>
-        <configuration>
-          <delimiters>
-            <delimiter>@</delimiter>
-          </delimiters>
-          <useDefaultDelimiters>false</useDefaultDelimiters>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>com.samaxes.maven</groupId>
-        <artifactId>maven-minify-plugin</artifactId>
-        <version>1.3.5</version>
-        <executions>
-          <execution>
-            <id>default-minify</id>
-            <phase>generate-resources</phase>
-            <configuration>
-              <webappSourceDir>${basedir}/src/main/resources</webappSourceDir>
-              <cssSourceDir>css</cssSourceDir>
-              <cssSourceFiles>
-                <cssSourceFile>bootstrap-${bootstrap.version}.css</cssSourceFile>
-                <cssSourceFile>maven-base.css</cssSourceFile>
-                <cssSourceFile>maven-theme.css</cssSourceFile>
-                <cssSourceFile>prettify.css</cssSourceFile>
-              </cssSourceFiles>
-              <cssFinalFile>apache-maven-fluido-${project.version}.css</cssFinalFile>
-              <jsSourceDir>js</jsSourceDir>
-              <jsSourceFiles>
-                <jsSourceFile>jquery-${jquery.version}.js</jsSourceFile>
-                <jsSourceFile>bootstrap-${bootstrap.version}.js</jsSourceFile>
-                <jsSourceFile>prettify.js</jsSourceFile>
-                <jsSourceFile>fluido.js</jsSourceFile>
-              </jsSourceFiles>
-              <jsFinalFile>apache-maven-fluido-${project.version}.js</jsFinalFile>
-            </configuration>
-            <goals>
-              <goal>minify</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-  <profiles>
-    <profile>
-      <id>run-its</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-invoker-plugin</artifactId>
-            <configuration>
-              <debug>true</debug>
-              <projectsDirectory>src/it</projectsDirectory>
-              <cloneProjectsTo>${project.build.directory}/it</cloneProjectsTo>
-              <preBuildHookScript>setup</preBuildHookScript>
-              <postBuildHookScript>verify</postBuildHookScript>
-              <localRepositoryPath>${project.build.directory}/local-repo</localRepositoryPath>
-              <settingsFile>src/it/settings.xml</settingsFile>
-              <pomIncludes>
-                <pomInclude>*/pom.xml</pomInclude>
-              </pomIncludes>
-              <goals>
-                <goal>site</goal>
-              </goals>
-            </configuration>
-            <executions>
-              <execution>
-                <id>integration-test</id>
-                <goals>
-                  <goal>install</goal>
-                  <goal>integration-test</goal>
-                  <goal>verify</goal>
-                </goals>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <profile>
-      <id>reporting</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-resources-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>copy-sidebar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/sidebar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/sidebar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-topbar-inverse</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/topbar-inverse/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/topbar-inverse/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-10</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-10/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-10/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-13</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-13/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-13/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-14</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-14/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-14/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-14_sitesearch</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-14_sitesearch/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-14_sitesearch/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-15</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-15/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-15/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-16</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-16/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-16/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-17</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-17/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-17/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-21</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-21/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-21/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22_default</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22_default/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22_default/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-22_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-22_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-22_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-23</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-23/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-23/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-24</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-24/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-24/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-24_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-24_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-24_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-25</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-25/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-25/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-28</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-28/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-28/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-31</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-31/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-31/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-33</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-33/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-33/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-33_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-33_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-33_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-34</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-34/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-34/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-34_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-34_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-34_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-41</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-41/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-41/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-72</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-72/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-72/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-75</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-75/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-75/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-76</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-76/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-76/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-76_topbar</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-76_topbar/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-76_topbar/</outputDirectory>
-                </configuration>
-              </execution>
-              <execution>
-                <id>copy-mskins-85</id>
-                <phase>site</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <resources>
-                    <resource>
-                      <directory>${project.build.directory}/it/mskins-85/target/site/</directory>
-                    </resource>
-                  </resources>
-                  <outputDirectory>${project.build.directory}/site/mskins-85/</outputDirectory>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-      <reporting>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-invoker-plugin</artifactId>
-            <version>1.8</version>
-          </plugin>
-        </plugins>
-      </reporting>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
----------------------------------------------------------------------
diff --git a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml b/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
deleted file mode 100644
index 65791e8..0000000
--- a/src/main/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
-  <groupId>org.apache.maven.skins</groupId>
-  <artifactId>maven-fluido-skin</artifactId>
-  <versioning>
-    <release>1.5-HBASE</release>
-    <versions>
-      <version>1.5-HBASE</version>
-    </versions>
-    <lastUpdated>20151111033340</lastUpdated>
-  </versioning>
-</metadata>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/site.xml
----------------------------------------------------------------------
diff --git a/src/main/site/site.xml b/src/main/site/site.xml
deleted file mode 100644
index e038f91..0000000
--- a/src/main/site/site.xml
+++ /dev/null
@@ -1,131 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-/**
- * 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.
- */
--->
-
-<project xmlns="http://maven.apache.org/DECORATION/1.0.0"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xsi:schemaLocation="http://maven.apache.org/DECORATION/1.0.0 http://maven.apache.org/xsd/decoration-1.0.0.xsd">
-  <skin>
-    <groupId>org.apache.maven.skins</groupId>
-    <artifactId>maven-fluido-skin</artifactId>
-    <version>1.5-HBASE</version>
-  </skin>
-  <custom>
-    <fluidoSkin>
-      <topBarEnabled>true</topBarEnabled>
-      <sideBarEnabled>false</sideBarEnabled>
-      <googleSearch>
-        <!-- The ID of the Google custom search engine to use.
-             This one searches hbase.apache.org, issues.apache.org/browse/HBASE-*,
-             and user and dev mailing list archives. -->
-        <customSearch>000385458301414556862:sq1bb0xugjg</customSearch>
-      </googleSearch>
-      <sourceLineNumbersEnabled>false</sourceLineNumbersEnabled>
-      <skipGenerationDate>true</skipGenerationDate>
-      <breadcrumbDivider>»</breadcrumbDivider>
-    </fluidoSkin>
-  </custom>
-  <bannerLeft>
-    <name>HBaseCon2017</name>
-    <src>images/hbaseconasia2017.png</src>
-    <href>https://www.eventbrite.com/e/hbasecon-asia-2017-tickets-34935546159</href>
-    <!--
-    <name/>
-    <height>0</height>
-    <width>0</width>
--->
-  </bannerLeft>
-  <bannerRight>
-    <name>Apache HBase</name>
-    <src>images/hbase_logo_with_orca_large.png</src>
-    <href>http://hbase.apache.org/</href>
-  </bannerRight>
-  <publishDate position="bottom"/>
-  <version position="none"/>
-  <body>
-    <head>
-      <meta name="viewport" content="width=device-width, initial-scale=1.0"></meta>
-      <link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap-responsive.min.css"/>
-      <link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.9.1/styles/github.min.css"/>
-      <link rel="stylesheet" href="css/site.css"/>
-      <script src="https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.9.1/highlight.min.js"></script>
-    </head>
-    <menu name="Apache HBase Project">
-      <item name="Overview" href="index.html"/>
-      <item name="License" href="license.html"/>
-      <item name="Downloads" href="http://www.apache.org/dyn/closer.cgi/hbase/"/>
-      <item name="Release Notes" href="https://issues.apache.org/jira/browse/HBASE?report=com.atlassian.jira.plugin.system.project:changelog-panel#selectedTab=com.atlassian.jira.plugin.system.project%3Achangelog-panel" />
-      <item name="Code Of Conduct" href="coc.html"/>
-      <item name="Blog" href="http://blogs.apache.org/hbase/"/>
-      <item name="Mailing Lists" href="mail-lists.html"/>
-      <item name="Team" href="team-list.html"/>
-      <item name="ReviewBoard" href="https://reviews.apache.org/"/>
-      <item name="Thanks" href="sponsors.html"/>
-      <item name="Powered by HBase" href="poweredbyhbase.html"/>
-      <item name="Other resources" href="resources.html"/>
-    </menu>
-    <menu name="Project Information">
-      <item name="Project Summary" href="project-summary.html"/>
-      <item name="Dependency Information" href="dependency-info.html"/>
-      <item name="Team" href="team-list.html"/>
-      <item name="Source Repository" href="source-repository.html"/>
-      <item name="Issue Tracking" href="issue-tracking.html"/>
-      <item name="Dependency Management" href="dependency-management.html"/>
-      <item name="Dependencies" href="dependencies.html"/>
-      <item name="Dependency Convergence" href="dependency-convergence.html"/>
-      <item name="Continuous Integration" href="integration.html"/>
-      <item name="Plugin Management" href="plugin-management.html"/>
-      <item name="Plugins" href="plugins.html"/>
-    </menu>
-    <menu name="Documentation and API">
-      <item name="Reference Guide" href="book.html" target="_blank" />
-      <item name="Reference Guide (PDF)" href="apache_hbase_reference_guide.pdf" target="_blank" />
-      <item name="Getting Started" href="book.html#quickstart" target="_blank" />
-      <item name="User API" href="apidocs/index.html" target="_blank" />
-      <item name="User API (Test)" href="testapidocs/index.html" target="_blank" />
-      <item name="Developer API" href="devapidocs/index.html" target="_blank" />
-      <item name="Developer API (Test)" href="testdevapidocs/index.html" target="_blank" />
-      <item name="中文参考指南(单页)" href="http://abloz.com/hbase/book.html" target="_blank" />
-      <item name="FAQ" href="book.html#faq" target="_blank" />
-      <item name="Videos/Presentations" href="book.html#other.info" target="_blank" />
-      <item name="Wiki" href="http://wiki.apache.org/hadoop/Hbase" target="_blank" />
-      <item name="ACID Semantics" href="acid-semantics.html" target="_blank" />
-      <item name="Bulk Loads" href="book.html#arch.bulk.load" target="_blank" />
-      <item name="Metrics" href="metrics.html" target="_blank" />
-      <item name="HBase on Windows" href="cygwin.html" target="_blank" />
-      <item name="Cluster replication" href="book.html#replication" target="_blank" />
-      <item name="1.2 Documentation">
-        <item name="API" href="1.2/apidocs/index.html" target="_blank" />
-        <item name="X-Ref" href="1.2/xref/index.html" target="_blank" />
-        <item name="Ref Guide (single-page)" href="1.2/book.html" target="_blank" />
-      </item>
-      <item name="1.1 Documentation">
-        <item name="API" href="1.1/apidocs/index.html" target="_blank" />
-        <item name="X-Ref" href="1.1/xref/index.html" target="_blank" />
-        <item name="Ref Guide (single-page)" href="1.1/book.html" target="_blank" />
-      </item>
-    </menu>
-    <menu name="ASF">
-      <item name="Apache Software Foundation" href="http://www.apache.org/foundation/" target="_blank" />
-      <item name="How Apache Works" href="http://www.apache.org/foundation/how-it-works.html" target="_blank" />
-      <item name="Sponsoring Apache" href="http://www.apache.org/foundation/sponsorship.html" target="_blank" />
-    </menu>
-    </body>
-</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/acid-semantics.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/acid-semantics.xml b/src/main/site/xdoc/acid-semantics.xml
deleted file mode 100644
index 2d4eb6a..0000000
--- a/src/main/site/xdoc/acid-semantics.xml
+++ /dev/null
@@ -1,235 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title> 
-      Apache HBase (TM) ACID Properties
-    </title>
-  </properties>
-
-  <body>
-    <section name="About this Document">
-      <p>Apache HBase (TM) is not an ACID compliant database. However, it does guarantee certain specific
-      properties.</p>
-      <p>This specification enumerates the ACID properties of HBase.</p>
-    </section>
-    <section name="Definitions">
-      <p>For the sake of common vocabulary, we define the following terms:</p>
-      <dl>
-        <dt>Atomicity</dt>
-        <dd>an operation is atomic if it either completes entirely or not at all</dd>
-
-        <dt>Consistency</dt>
-        <dd>
-          all actions cause the table to transition from one valid state directly to another
-          (eg a row will not disappear during an update, etc)
-        </dd>
-
-        <dt>Isolation</dt>
-        <dd>
-          an operation is isolated if it appears to complete independently of any other concurrent transaction
-        </dd>
-
-        <dt>Durability</dt>
-        <dd>any update that reports &quot;successful&quot; to the client will not be lost</dd>
-
-        <dt>Visibility</dt>
-        <dd>an update is considered visible if any subsequent read will see the update as having been committed</dd>
-      </dl>
-      <p>
-        The terms <em>must</em> and <em>may</em> are used as specified by RFC 2119.
-        In short, the word &quot;must&quot; implies that, if some case exists where the statement
-        is not true, it is a bug. The word &quot;may&quot; implies that, even if the guarantee
-        is provided in a current release, users should not rely on it.
-      </p>
-    </section>
-    <section name="APIs to consider">
-      <ul>
-        <li>Read APIs
-        <ul>
-          <li>get</li>
-          <li>scan</li>
-        </ul>
-        </li>
-        <li>Write APIs</li>
-        <ul>
-          <li>put</li>
-          <li>batch put</li>
-          <li>delete</li>
-        </ul>
-        <li>Combination (read-modify-write) APIs</li>
-        <ul>
-          <li>incrementColumnValue</li>
-          <li>checkAndPut</li>
-        </ul>
-      </ul>
-    </section>
-
-    <section name="Guarantees Provided">
-
-      <section name="Atomicity">
-
-        <ol>
-          <li>All mutations are atomic within a row. Any put will either wholly succeed or wholly fail.[3]</li>
-          <ol>
-            <li>An operation that returns a &quot;success&quot; code has completely succeeded.</li>
-            <li>An operation that returns a &quot;failure&quot; code has completely failed.</li>
-            <li>An operation that times out may have succeeded and may have failed. However,
-            it will not have partially succeeded or failed.</li>
-          </ol>
-          <li> This is true even if the mutation crosses multiple column families within a row.</li>
-          <li> APIs that mutate several rows will _not_ be atomic across the multiple rows.
-          For example, a multiput that operates on rows 'a','b', and 'c' may return having
-          mutated some but not all of the rows. In such cases, these APIs will return a list
-          of success codes, each of which may be succeeded, failed, or timed out as described above.</li>
-          <li> The checkAndPut API happens atomically like the typical compareAndSet (CAS) operation
-          found in many hardware architectures.</li>
-          <li> The order of mutations is seen to happen in a well-defined order for each row, with no
-          interleaving. For example, if one writer issues the mutation &quot;a=1,b=1,c=1&quot; and
-          another writer issues the mutation &quot;a=2,b=2,c=2&quot;, the row must either
-          be &quot;a=1,b=1,c=1&quot; or &quot;a=2,b=2,c=2&quot; and must <em>not</em> be something
-          like &quot;a=1,b=2,c=1&quot;.</li>
-          <ol>
-            <li>Please note that this is not true _across rows_ for multirow batch mutations.</li>
-          </ol>
-        </ol>
-      </section>
-      <section name="Consistency and Isolation">
-        <ol>
-          <li>All rows returned via any access API will consist of a complete row that existed at
-          some point in the table's history.</li>
-          <li>This is true across column families - i.e a get of a full row that occurs concurrent
-          with some mutations 1,2,3,4,5 will return a complete row that existed at some point in time
-          between mutation i and i+1 for some i between 1 and 5.</li>
-          <li>The state of a row will only move forward through the history of edits to it.</li>
-        </ol>
-
-        <section name="Consistency of Scans">
-        <p>
-          A scan is <strong>not</strong> a consistent view of a table. Scans do
-          <strong>not</strong> exhibit <em>snapshot isolation</em>.
-        </p>
-        <p>
-          Rather, scans have the following properties:
-        </p>
-
-        <ol>
-          <li>
-            Any row returned by the scan will be a consistent view (i.e. that version
-            of the complete row existed at some point in time) [1]
-          </li>
-          <li>
-            A scan will always reflect a view of the data <em>at least as new as</em>
-            the beginning of the scan. This satisfies the visibility guarantees
-          enumerated below.</li>
-          <ol>
-            <li>For example, if client A writes data X and then communicates via a side
-            channel to client B, any scans started by client B will contain data at least
-            as new as X.</li>
-            <li>A scan _must_ reflect all mutations committed prior to the construction
-            of the scanner, and _may_ reflect some mutations committed subsequent to the
-            construction of the scanner.</li>
-            <li>Scans must include <em>all</em> data written prior to the scan (except in
-            the case where data is subsequently mutated, in which case it _may_ reflect
-            the mutation)</li>
-          </ol>
-        </ol>
-        <p>
-          Those familiar with relational databases will recognize this isolation level as &quot;read committed&quot;.
-        </p>
-        <p>
-          Please note that the guarantees listed above regarding scanner consistency
-          are referring to &quot;transaction commit time&quot;, not the &quot;timestamp&quot;
-          field of each cell. That is to say, a scanner started at time <em>t</em> may see edits
-          with a timestamp value greater than <em>t</em>, if those edits were committed with a
-          &quot;forward dated&quot; timestamp before the scanner was constructed.
-        </p>
-        </section>
-      </section>
-      <section name="Visibility">
-        <ol>
-          <li> When a client receives a &quot;success&quot; response for any mutation, that
-          mutation is immediately visible to both that client and any client with whom it
-          later communicates through side channels. [3]</li>
-          <li> A row must never exhibit so-called &quot;time-travel&quot; properties. That
-          is to say, if a series of mutations moves a row sequentially through a series of
-          states, any sequence of concurrent reads will return a subsequence of those states.</li>
-          <ol>
-            <li>For example, if a row's cells are mutated using the &quot;incrementColumnValue&quot;
-            API, a client must never see the value of any cell decrease.</li>
-            <li>This is true regardless of which read API is used to read back the mutation.</li>
-          </ol>
-          <li> Any version of a cell that has been returned to a read operation is guaranteed to
-          be durably stored.</li>
-        </ol>
-
-      </section>
-      <section name="Durability">
-        <ol>
-          <li> All visible data is also durable data. That is to say, a read will never return
-          data that has not been made durable on disk[2]</li>
-          <li> Any operation that returns a &quot;success&quot; code (eg does not throw an exception)
-          will be made durable.[3]</li>
-          <li> Any operation that returns a &quot;failure&quot; code will not be made durable
-          (subject to the Atomicity guarantees above)</li>
-          <li> All reasonable failure scenarios will not affect any of the guarantees of this document.</li>
-
-        </ol>
-      </section>
-      <section name="Tunability">
-        <p>All of the above guarantees must be possible within Apache HBase. For users who would like to trade
-        off some guarantees for performance, HBase may offer several tuning options. For example:</p>
-        <ul>
-          <li>Visibility may be tuned on a per-read basis to allow stale reads or time travel.</li>
-          <li>Durability may be tuned to only flush data to disk on a periodic basis</li>
-        </ul>
-      </section>
-    </section>
-    <section name="More Information">
-      <p>
-      For more information, see the <a href="book.html#client">client architecture</a> or <a href="book.html#datamodel">data model</a> sections in the Apache HBase Reference Guide. 
-      </p>
-    </section>
-    
-    <section name="Footnotes">
-      <p>[1] A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of
-          a row in one RPC then going back to fetch another portion of the row in a subsequent RPC.
-          Intra-row scanning happens when you set a limit on how many values to return per Scan#next
-          (See <a href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)">Scan#setBatch(int)</a>).
-      </p>
-
-      <p>[2] In the context of Apache HBase, &quot;durably on disk&quot; implies an hflush() call on the transaction
-      log. This does not actually imply an fsync() to magnetic media, but rather just that the data has been
-      written to the OS cache on all replicas of the log. In the case of a full datacenter power loss, it is
-      possible that the edits are not truly durable.</p>
-      <p>[3] Puts will either wholly succeed or wholly fail, provided that they are actually sent
-      to the RegionServer.  If the writebuffer is used, Puts will not be sent until the writebuffer is filled
-      or it is explicitly flushed.</p>
-      
-    </section>
-
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/bulk-loads.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/bulk-loads.xml b/src/main/site/xdoc/bulk-loads.xml
deleted file mode 100644
index 2195003..0000000
--- a/src/main/site/xdoc/bulk-loads.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title> 
-      Bulk Loads in Apache HBase (TM)
-    </title>
-  </properties>
-  <body>
-       <p>This page has been retired.  The contents have been moved to the 
-      <a href="http://hbase.apache.org/book.html#arch.bulk.load">Bulk Loading</a> section
- in the Reference Guide.
- </p>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/coc.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/coc.xml b/src/main/site/xdoc/coc.xml
deleted file mode 100644
index fc2b549..0000000
--- a/src/main/site/xdoc/coc.xml
+++ /dev/null
@@ -1,92 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>
-      Code of Conduct Policy
-    </title>
-  </properties>
-  <body>
-  <section name="Code of Conduct Policy">
-<p>
-We expect participants in discussions on the HBase project mailing lists, IRC
-channels, and JIRA issues to abide by the Apache Software Foundation's
-<a href="http://apache.org/foundation/policies/conduct.html">Code of Conduct</a>.
-</p>
-<p>
-If you feel there has been a violation of this code, please point out your
-concerns publicly in a friendly and matter of fact manner. Nonverbal
-communication is prone to misinterpretation and misunderstanding. Everyone has
-bad days and sometimes says things they regret later. Someone else's
-communication style may clash with yours, but the difference can be amicably
-resolved. After pointing out your concerns please be generous upon receiving an
-apology.
-</p>
-<p>
-Should there be repeated instances of code of conduct violations, or if there is
-an obvious and severe violation, the HBase PMC may become involved. When this
-happens the PMC will openly discuss the matter, most likely on the dev@hbase
-mailing list, and will consider taking the following actions, in order, if there
-is a continuing problem with an individual:
-<ol>
-<li>A friendly off-list warning;</li>
-<li>A friendly public warning, if the communication at issue was on list, otherwise another off-list warning;</li>
-<li>A three month suspension from the public mailing lists and possible operator action in the IRC channels.</li>
-<li>A permanent ban from the public mailing lists, IRC channels, and project JIRA.</li>
-</ol>
-</p>
-<p>
-For flagrant violations requiring a firm response the PMC may opt to skip early
-steps. No action will be taken before public discussion leading to consensus or
-a successful majority vote.
-</p>
-  </section>
-  <section name="Diversity Statement">
-<p>
-As a project and a community, we encourage you to participate in the HBase project
-in whatever capacity suits you, whether it involves development, documentation,
-answering questions on mailing lists, triaging issue and patch review, managing
-releases, or any other way that you want to help. We appreciate your
-contributions and the time you dedicate to the HBase project. We strive to
-recognize the work of participants publicly. Please let us know if we can
-improve in this area.
-</p>
-<p>
-We value diversity and strive to support participation by people with all
-different backgrounds. Rich projects grow from groups with different points of
-view and different backgrounds. We welcome your suggestions about how we can
-welcome participation by people at all skill levels and with all aspects of the
-project.
-</p>
-<p>
-If you can think of something we are doing that we shouldn't, or something that
-we should do but aren't, please let us know. If you feel comfortable doing so,
-use the public mailing lists. Otherwise, reach out to a PMC member or send an
-email to <a href="mailto:private@hbase.apache.org">the private PMC mailing list</a>.
-</p>
-  </section>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/cygwin.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/cygwin.xml b/src/main/site/xdoc/cygwin.xml
deleted file mode 100644
index 406c0a9..0000000
--- a/src/main/site/xdoc/cygwin.xml
+++ /dev/null
@@ -1,245 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Installing Apache HBase (TM) on Windows using Cygwin</title>
-  </properties>
-
-<body>
-<section name="Introduction">
-<p><a title="HBase project" href="http://hbase.apache.org" target="_blank">Apache HBase (TM)</a> is a distributed, column-oriented store, modeled after Google's <a title="Google's BigTable" href="http://research.google.com/archive/bigtable.html" target="_blank">BigTable</a>. Apache HBase is built on top of <a title="Hadoop project" href="http://hadoop.apache.org">Hadoop</a> for its <a title="Hadoop MapReduce project" href="http://hadoop.apache.org/mapreduce" target="_blank">MapReduce </a>and <a title="Hadoop DFS project" href="http://hadoop.apache.org/hdfs">distributed file system</a> implementation. All these projects are open-source and part of the <a title="The Apache Software Foundation" href="http://www.apache.org/" target="_blank">Apache Software Foundation</a>.</p>
-
-<p style="text-align: justify; ">As being distributed, large scale platforms, the Hadoop and HBase projects mainly focus on <em><strong>*nix</strong></em><strong> environments</strong> for production installations. However, being developed in <strong>Java</strong>, both projects are fully <strong>portable</strong> across platforms and, hence, also to the <strong>Windows operating system</strong>. For ease of development the projects rely on <a title="Cygwin site" href="http://www.cygwin.com/" target="_blank">Cygwin</a> to have a *nix-like environment on Windows to run the shell scripts.</p>
-</section>
-<section name="Purpose">
-<p style="text-align: justify; ">This document explains the <strong>intricacies of running Apache HBase on Windows using Cygwin</strong> as an all-in-one single-node installation for testing and development. The HBase <a title="HBase Overview" href="http://hbase.apache.org/apidocs/overview-summary.html#overview_description" target="_blank">Overview</a> and <a title="HBase QuickStart" href="http://hbase.apache.org/book/quickstart.html" target="_blank">QuickStart</a> guides on the other hand go a long way in explaning how to setup <a title="HBase project" href="http://hadoop.apache.org/hbase" target="_blank">HBase</a> in more complex deployment scenario's.</p>
-</section>
-
-<section name="Installation">
-<p style="text-align: justify; ">For running Apache HBase on Windows, 3 technologies are required: <strong>Java, Cygwin and SSH</strong>. The following paragraphs detail the installation of each of the aforementioned technologies.</p>
-<section name="Java">
-<p style="text-align: justify; ">HBase depends on the <a title="Java Platform, Standard Edition, 6 Release" href="http://java.sun.com/javase/6/" target="_blank">Java Platform, Standard Edition, 6 Release</a>. So the target system has to be provided with at least the Java Runtime Environment (JRE); however if the system will also be used for development, the Jave Development Kit (JDK) is preferred. You can download the latest versions for both from <a title="Java SE Downloads" href="http://java.sun.com/javase/downloads/index.jsp" target="_blank">Sun's download page</a>. Installation is a simple GUI wizard that guides you through the process.</p>
-</section>
-<section name="Cygwin">
-<p style="text-align: justify; ">Cygwin is probably the oddest technology in this solution stack. It provides a dynamic link library that emulates most of a *nix environment on Windows. On top of that a whole bunch of the most common *nix tools are supplied. Combined, the DLL with the tools form a very *nix-alike environment on Windows.</p>
-
-<p style="text-align: justify; ">For installation, Cygwin provides the <a title="Cygwin Setup Utility" href="http://cygwin.com/setup.exe" target="_blank"><strong><code>setup.exe</code> utility</strong></a> that tracks the versions of all installed components on the target system and provides the mechanism for <strong>installing</strong> or <strong>updating </strong>everything from the mirror sites of Cygwin.</p>
-
-<p style="text-align: justify; ">To support installation, the <code>setup.exe</code> utility uses 2 directories on the target system. The <strong>Root</strong> directory for Cygwin (defaults to <code>C:\cygwin)</code> which will become <code>/</code> within the eventual Cygwin installation; and the <strong>Local Package </strong>directory (e.g. <code>C:\cygsetup</code> that is the cache where <code>setup.exe</code> stores the packages before they are installed. The cache must not be the same folder as the Cygwin root.</p>
-
-<p style="text-align: justify; ">Perform following steps to install Cygwin, which are elaboratly detailed in the <a title="Setting Up Cygwin" href="http://cygwin.com/cygwin-ug-net/setup-net.html" target="_self">2nd chapter</a> of the <a title="Cygwin User's Guide" href="http://cygwin.com/cygwin-ug-net/cygwin-ug-net.html" target="_blank">Cygwin User's Guide</a>:</p>
-
-<ol style="text-align: justify; ">
-	<li>Make sure you have <code>Administrator</code> privileges on the target system.</li>
-	<li>Choose and create you <strong>Root</strong> and <strong>Local Package</strong> directories. A good suggestion is to use <code>C:\cygwin\root</code> and <code>C:\cygwin\setup</code> folders.</li>
-	<li>Download the <code>setup.exe</code> utility and save it to the <strong>Local Package</strong> directory.</li>
-	<li>Run the <code>setup.exe</code> utility,
-<ol>
-	<li>Choose  the <code>Install from Internet</code> option,</li>
-	<li>Choose your <strong>Root</strong> and <strong>Local Package</strong> folders</li>
-	<li>and select an appropriate mirror.</li>
-	<li>Don't select any additional packages yet, as we only want to install Cygwin for now.</li>
-	<li>Wait for download and install</li>
-	<li>Finish the installation</li>
-</ol>
-</li>
-	<li>Optionally, you can now also add a shortcut to your Start menu pointing to the <code>setup.exe</code> utility in the <strong>Local Package </strong>folder.</li>
-	<li>Add <code>CYGWIN_HOME</code> system-wide environment variable that points to your <strong>Root </strong>directory.</li>
-	<li>Add <code>%CYGWIN_HOME%\bin</code> to the end of your <code>PATH</code> environment variable.</li>
-	<li>Reboot the sytem after making changes to the environment variables otherwise the OS will not be able to find the Cygwin utilities.</li>
-	<li>Test your installation by running your freshly created shortcuts or the <code>Cygwin.bat</code> command in the <strong>Root</strong> folder. You should end up in a terminal window that is running a <a title="Bash Reference Manual" href="http://www.gnu.org/software/bash/manual/bashref.html" target="_blank">Bash shell</a>. Test the shell by issuing following commands:
-<ol>
-	<li><code>cd /</code> should take you to thr <strong>Root</strong> directory in Cygwin;</li>
-	<li>the <code>LS</code> commands that should list all files and folders in the current directory.</li>
-	<li>Use the <code>exit</code> command to end the terminal.</li>
-</ol>
-</li>
-	<li>When needed, to <strong>uninstall</strong> Cygwin you can simply delete the <strong>Root</strong> and <strong>Local Package</strong> directory, and the <strong>shortcuts</strong> that were created during installation.</li>
-</ol>
-</section>
-<section name="SSH">
-<p style="text-align: justify; ">HBase (and Hadoop) rely on <a title="Secure Shell" href="http://nl.wikipedia.org/wiki/Secure_Shell" target="_blank"><strong>SSH</strong></a> for interprocess/-node <strong>communication</strong> and launching<strong> remote commands</strong>. SSH will be provisioned on the target system via Cygwin, which supports running Cygwin programs as <strong>Windows services</strong>!</p>
-
-<ol style="text-align: justify; ">
-	<li>Rerun the <code><strong>setup.exe</strong></code><strong> utility</strong>.</li>
-	<li>Leave all parameters as is, skipping through the wizard using the <code>Next</code> button until the <code>Select Packages</code> panel is shown.</li>
-	<li>Maximize the window and click the <code>View</code> button to toggle to the list view, which is ordered alfabetically on <code>Package</code>, making it easier to find the packages we'll need.</li>
-	<li>Select the following packages by clicking the status word (normally <code>Skip</code>) so it's marked for installation. Use the <code>Next </code>button to download and install the packages.
-<ol>
-	<li>OpenSSH</li>
-	<li>tcp_wrappers</li>
-	<li>diffutils</li>
-	<li>zlib</li>
-</ol>
-</li>
-	<li>Wait for the install to complete and finish the installation.</li>
-</ol>
-</section>
-<section name="HBase">
-<p style="text-align: justify; ">Download the <strong>latest release </strong>of Apache HBase from the <a title="HBase Releases" href="http://www.apache.org/dyn/closer.cgi/hbase/" target="_blank">website</a>. As the Apache HBase distributable is just a zipped archive, installation is as simple as unpacking the archive so it ends up in its final <strong>installation</strong> directory. Notice that HBase has to be installed in Cygwin and a good directory suggestion is to use <code>/usr/local/</code> (or [<code><strong>Root</strong> directory]\usr\local</code> in Windows slang). You should end up with a <code>/usr/local/hbase-<em>&lt;version&gt;</em></code> installation in Cygwin.</p>
-
-This finishes installation. We go on with the configuration.
-</section>
-</section>
-<section name="Configuration">
-<p style="text-align: justify; ">There are 3 parts left to configure: <strong>Java, SSH and HBase</strong> itself. Following paragraphs explain eacht topic in detail.</p>
-<section name="Java">
-<p style="text-align: justify; ">One important thing to remember in shell scripting in general (i.e. *nix and Windows) is that managing, manipulating and assembling path names that contains spaces can be very hard, due to the need to escape and quote those characters and strings. So we try to stay away from spaces in path names. *nix environments can help us out here very easily by using <strong>symbolic links</strong>.</p>
-
-<ol style="text-align: justify; ">
-	<li style="text-align: justify; ">Create a link in <code>/usr/local</code> to the Java home directory by using the following command and substituting the name of your chosen Java environment:
-<pre>LN -s /cygdrive/c/Program\ Files/Java/<em>&lt;jre name&gt; </em>/usr/local/<em>&lt;jre name&gt;</em></pre>
-</li>
-	<li>Test your java installation by changing directories to your Java folder <code>CD /usr/local/<em>&lt;jre name&gt;</em></code> and issueing the command <code>./bin/java -version</code>. This should output your version of the chosen JRE.</li>
-</ol>
-</section>
-<section>
-<title>SSH</title>
-<p style="text-align: justify; ">Configuring <strong>SSH </strong>is quite elaborate, but primarily a question of launching it by default as a<strong> Windows service</strong>.</p>
-
-<ol style="text-align: justify; ">
-	<li style="text-align: justify; ">On Windows Vista and above make sure you run the Cygwin shell with <strong>elevated privileges</strong>, by right-clicking on the shortcut an using <code>Run as Administrator</code>.</li>
-	<li style="text-align: justify; ">First of all, we have to make sure the <strong>rights on some crucial files</strong> are correct. Use the commands underneath. You can verify all rights by using the <code>LS -L</code> command on the different files. Also, notice the auto-completion feature in the shell using <code>&lt;TAB&gt;</code> is extremely handy in these situations.
-<ol>
-	<li><code>chmod +r /etc/passwd</code> to make the passwords file readable for all</li>
-	<li><code>chmod u+w /etc/passwd</code> to make the passwords file writable for the owner</li>
-	<li><code>chmod +r /etc/group</code> to make the groups file readable for all</li>
-</ol>
-<ol>
-	<li><code>chmod u+w /etc/group</code> to make the groups file writable for the owner</li>
-</ol>
-<ol>
-	<li><code>chmod 755 /var</code> to make the var folder writable to owner and readable and executable to all</li>
-</ol>
-</li>
-	<li>Edit the <strong>/etc/hosts.allow</strong> file using your favorite editor (why not VI in the shell!) and make sure the following two lines are in there before the <code>PARANOID</code> line:
-<ol>
-	<li><code>ALL : localhost 127.0.0.1/32 : allow</code></li>
-	<li><code>ALL : [::1]/128 : allow</code></li>
-</ol>
-</li>
-	<li>Next we have to <strong>configure SSH</strong> by using the script <code>ssh-host-config</code>
-<ol>
-	<li>If this script asks to overwrite an existing <code>/etc/ssh_config</code>, answer <code>yes</code>.</li>
-	<li>If this script asks to overwrite an existing <code>/etc/sshd_config</code>, answer <code>yes</code>.</li>
-	<li>If this script asks to use privilege separation, answer <code>yes</code>.</li>
-	<li>If this script asks to install <code>sshd</code> as a service, answer <code>yes</code>. Make sure you started your shell as Adminstrator!</li>
-	<li>If this script asks for the CYGWIN value, just <code>&lt;enter&gt;</code> as the default is <code>ntsec</code>.</li>
-	<li>If this script asks to create the <code>sshd</code> account, answer <code>yes</code>.</li>
-	<li>If this script asks to use a different user name as service account, answer <code>no</code> as the default will suffice.</li>
-	<li>If this script asks to create the <code>cyg_server</code> account, answer <code>yes</code>. Enter a password for the account.</li>
-</ol>
-</li>
-	<li><strong>Start the SSH service</strong> using <code>net start sshd</code> or <code>cygrunsrv  --start  sshd</code>. Notice that <code>cygrunsrv</code> is the utility that make the process run as a Windows service. Confirm that you see a message stating that <code>the CYGWIN sshd service  was started succesfully.</code></li>
-	<li>Harmonize Windows and Cygwin<strong> user account</strong> by using the commands:
-<ol>
-	<li><code>mkpasswd -cl &gt; /etc/passwd</code></li>
-	<li><code>mkgroup --local &gt; /etc/group</code></li>
-</ol>
-</li>
-	<li><strong>Test </strong>the installation of SSH:
-<ol>
-	<li>Open a new Cygwin terminal</li>
-	<li>Use the command <code>whoami</code> to verify your userID</li>
-	<li>Issue an <code>ssh localhost</code> to connect to the system itself
-<ol>
-	<li>Answer <code>yes</code> when presented with the server's fingerprint</li>
-	<li>Issue your password when prompted</li>
-	<li>test a few commands in the remote session</li>
-	<li>The <code>exit</code> command should take you back to your first shell in Cygwin</li>
-</ol>
-</li>
-	<li><code>Exit</code> should terminate the Cygwin shell.</li>
-</ol>
-</li>
-</ol>
-</section>
-<section name="HBase">
-If all previous configurations are working properly, we just need some tinkering at the <strong>HBase config</strong> files to properly resolve on Windows/Cygwin. All files and paths referenced here start from the HBase <code>[<strong>installation</strong> directory]</code> as working directory.
-<ol>
-	<li>HBase uses the <code>./conf/<strong>hbase-env.sh</strong></code> to configure its dependencies on the runtime environment. Copy and uncomment following lines just underneath their original, change them to fit your environemnt. They should read something like:
-<ol>
-	<li><code>export JAVA_HOME=/usr/local/<em>&lt;jre name&gt;</em></code></li>
-	<li><code>export HBASE_IDENT_STRING=$HOSTNAME</code> as this most likely does not inlcude spaces.</li>
-</ol>
-</li>
-	<li>HBase uses the ./conf/<code><strong>hbase-default.xml</strong></code> file for configuration. Some properties do not resolve to existing directories because the JVM runs on Windows. This is the major issue to keep in mind when working with Cygwin: within the shell all paths are *nix-alike, hence relative to the root <code>/</code>. However, every parameter that is to be consumed within the windows processes themself, need to be Windows settings, hence <code>C:\</code>-alike. Change following propeties in the configuration file, adjusting paths where necessary to conform with your own installation:
-<ol>
-	<li><code>hbase.rootdir</code> must read e.g. <code>file:///C:/cygwin/root/tmp/hbase/data</code></li>
-	<li><code>hbase.tmp.dir</code> must read <code>C:/cygwin/root/tmp/hbase/tmp</code></li>
-	<li><code>hbase.zookeeper.quorum</code> must read <code>127.0.0.1</code> because for some reason <code>localhost</code> doesn't seem to resolve properly on Cygwin.</li>
-</ol>
-</li>
-	<li>Make sure the configured <code>hbase.rootdir</code> and <code>hbase.tmp.dir</code> <strong>directories exist</strong> and have the proper<strong> rights</strong> set up e.g. by issuing a <code>chmod 777</code> on them.</li>
-</ol>
-</section>
-</section>
-<section>
-<title>Testing</title>
-<p>
-This should conclude the installation and configuration of Apache HBase on Windows using Cygwin. So it's time <strong>to test it</strong>.
-<ol>
-	<li>Start a Cygwin<strong> terminal</strong>, if you haven't already.</li>
-	<li>Change directory to HBase <strong>installation</strong> using <code>CD /usr/local/hbase-<em>&lt;version&gt;</em></code>, preferably using auto-completion.</li>
-	<li><strong>Start HBase</strong> using the command <code>./bin/start-hbase.sh</code>
-<ol>
-	<li>When prompted to accept the SSH fingerprint, answer <code>yes</code>.</li>
-	<li>When prompted, provide your password. Maybe multiple times.</li>
-	<li>When the command completes, the HBase server should have started.</li>
-	<li>However, to be absolutely certain, check the logs in the <code>./logs</code> directory for any exceptions.</li>
-</ol>
-</li>
-	<li>Next we <strong>start the HBase shell</strong> using the command <code>./bin/hbase shell</code></li>
-	<li>We run some simple <strong>test commands</strong>
-<ol>
-	<li>Create a simple table using command <code>create 'test', 'data'</code></li>
-	<li>Verify the table exists using the command <code>list</code></li>
-	<li>Insert data into the table using e.g.
-<pre>put 'test', 'row1', 'data:1', 'value1'
-put 'test', 'row2', 'data:2', 'value2'
-put 'test', 'row3', 'data:3', 'value3'</pre>
-</li>
-	<li>List all rows in the table using the command <code>scan 'test'</code> that should list all the rows previously inserted. Notice how 3 new columns where added without changing the schema!</li>
-	<li>Finally we get rid of the table by issuing <code>disable 'test'</code> followed by <code>drop 'test'</code> and verified by <code>list</code> which should give an empty listing.</li>
-</ol>
-</li>
-	<li><strong>Leave the shell</strong> by <code>exit</code></li>
-	<li>To <strong>stop the HBase server</strong> issue the <code>./bin/stop-hbase.sh</code> command. And wait for it to complete!!! Killing the process might corrupt your data on disk.</li>
-	<li>In case of <strong>problems</strong>,
-<ol>
-	<li>verify the HBase logs in the <code>./logs</code> directory.</li>
-	<li>Try to fix the problem</li>
-	<li>Get help on the forums or IRC (<code>#hbase@freenode.net</code>). People are very active and keen to help out!</li>
-	<li>Stopr, restart and retest the server.</li>
-</ol>
-</li>
-</ol>
-</p>
-</section>
-
-<section name="Conclusion">
-<p>
-Now your <strong>HBase </strong>server is running, <strong>start coding</strong> and build that next killer app on this particular, but scalable datastore!
-</p>
-</section>
-</body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/export_control.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/export_control.xml b/src/main/site/xdoc/export_control.xml
deleted file mode 100644
index 0fd5c4f..0000000
--- a/src/main/site/xdoc/export_control.xml
+++ /dev/null
@@ -1,59 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>
-      Export Control
-    </title>
-  </properties>
-  <body>
-  <section name="Export Control">
-<p>
-This distribution uses or includes cryptographic software. The country in
-which you currently reside may have restrictions on the import, possession,
-use, and/or re-export to another country, of encryption software. BEFORE
-using any encryption software, please check your country's laws, regulations
-and policies concerning the import, possession, or use, and re-export of
-encryption software, to see if this is permitted. See the
-<a href="http://www.wassenaar.org/">Wassenaar Arrangement</a> for more
-information.</p>
-<p>
-The U.S. Government Department of Commerce, Bureau of Industry and Security 
-(BIS), has classified this software as Export Commodity Control Number (ECCN) 
-5D002.C.1, which includes information security software using or performing 
-cryptographic functions with asymmetric algorithms. The form and manner of this
-Apache Software Foundation distribution makes it eligible for export under the 
-License Exception ENC Technology Software Unrestricted (TSU) exception (see the
-BIS Export Administration Regulations, Section 740.13) for both object code and
-source code.</p>
-<p>
-Apache HBase uses the built-in java cryptography libraries. See Oracle's
-information regarding
-<a href="http://www.oracle.com/us/products/export/export-regulations-345813.html">Java cryptographic export regulations</a>
-for more details.</p>
-  </section>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/index.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/index.xml b/src/main/site/xdoc/index.xml
deleted file mode 100644
index 1848d40..0000000
--- a/src/main/site/xdoc/index.xml
+++ /dev/null
@@ -1,109 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Apache HBase&#8482; Home</title>
-    <link rel="shortcut icon" href="/images/favicon.ico" />
-  </properties>
-
-  <body>
-    <section name="Welcome to Apache HBase&#8482;">
-        <p><a href="http://www.apache.org/">Apache</a> HBase&#8482; is the <a href="http://hadoop.apache.org/">Hadoop</a> database, a distributed, scalable, big data store.
-    </p>
-    <p>Use Apache HBase&#8482; when you need random, realtime read/write access to your Big Data.
-    This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware.
-Apache HBase is an open-source, distributed, versioned, non-relational database modeled after Google's <a href="http://research.google.com/archive/bigtable.html">Bigtable: A Distributed Storage System for Structured Data</a> by Chang et al.
- Just as Bigtable leverages the distributed data storage provided by the Google File System, Apache HBase provides Bigtable-like capabilities on top of Hadoop and HDFS.
-    </p>
-  </section>
-    <section name="Download">
-    <p>
-    Click <b><a href="http://www.apache.org/dyn/closer.cgi/hbase/">here</a></b> to download Apache HBase&#8482;.
-    </p>
-    </section>
-    <section name="Features">
-    <p>
-<ul>
-    <li>Linear and modular scalability.
-</li>
-    <li>Strictly consistent reads and writes.
-</li>
-    <li>Automatic and configurable sharding of tables
-</li>
-    <li>Automatic failover support between RegionServers.
-</li>
-    <li>Convenient base classes for backing Hadoop MapReduce jobs with Apache HBase tables.
-</li>
-    <li>Easy to use Java API for client access.
-</li>
-    <li>Block cache and Bloom Filters for real-time queries.
-</li>
-    <li>Query predicate push down via server side Filters
-</li>
-    <li>Thrift gateway and a REST-ful Web service that supports XML, Protobuf, and binary data encoding options
-</li>
-    <li>Extensible jruby-based (JIRB) shell
-</li>
-    <li>Support for exporting metrics via the Hadoop metrics subsystem to files or Ganglia; or via JMX
-</li>
-</ul>
-</p>
-</section>
-     <section name="More Info">
-   <p>See the <a href="http://hbase.apache.org/book.html#arch.overview">Architecture Overview</a>, the <a href="http://hbase.apache.org/book.html#faq">Apache HBase Reference Guide FAQ</a>,
-    and the other documentation links.
-   </p>
-   <dl>
-     <dt>Export Control</dt>
-   <dd><p>The HBase distribution includes cryptographic software. See the export control notice <a href="export_control.html">here</a>
-   </p></dd>
-     <dt>Code Of Conduct</dt>
-   <dd><p>We expect participants in discussions on the HBase project mailing lists, Slack and IRC channels, and JIRA issues to abide by the Apache Software Foundation's <a href="http://apache.org/foundation/policies/conduct.html">Code of Conduct</a>. More information can be found <a href="coc.html">here</a>.
-   </p></dd>
- </dl>
-</section>
-
-     <section name="News">
-       <p>August 4th, 2017 <a href="https://easychair.org/cfp/HBaseConAsia2017">HBaseCon Asia 2017</a> @ the Huawei Campus in Shenzhen, China</p>
-       <p>June 12th, 2017 <a href="https://easychair.org/cfp/hbasecon2017">HBaseCon2017</a> at the Crittenden Buildings on the Google Mountain View Campus</p>
-       <p>April 25th, 2017 <a href="https://www.meetup.com/hbaseusergroup/events/239291716/">Meetup</a> @ Visa in Palo Alto</p>
-        <p>December 8th, 2016 <a href="https://www.meetup.com/hbaseusergroup/events/235542241/">Meetup@Splice</a> in San Francisco</p>
-       <p>September 26th, 2016 <a href="http://www.meetup.com/HBase-NYC/events/233024937/">HBaseConEast2016</a> at Google in Chelsea, NYC</p>
-         <p>May 24th, 2016 <a href="http://www.hbasecon.com/">HBaseCon2016</a> at The Village, 969 Market, San Francisco</p>
-       <p>June 25th, 2015 <a href="http://www.zusaar.com/event/14057003">HBase Summer Meetup 2015</a> in Tokyo</p>
-       <p>May 7th, 2015 <a href="http://hbasecon.com/">HBaseCon2015</a> in San Francisco</p>
-       <p>February 17th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/219260093/">HBase meetup around Strata+Hadoop World</a> in San Jose</p>
-       <p>January 15th, 2015 <a href="http://www.meetup.com/hbaseusergroup/events/218744798/">HBase meetup @ AppDynamics</a> in San Francisco</p>
-       <p>November 20th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/205219992/">HBase meetup @ WANdisco</a> in San Ramon</p>
-       <p>October 27th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/207386102/">HBase Meetup @ Apple</a> in Cupertino</p>
-       <p>October 15th, 2014 <a href="http://www.meetup.com/HBase-NYC/events/207655552/">HBase Meetup @ Google</a> on the night before Strata/HW in NYC</p>
-       <p>September 25th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/203173692/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
-         <p>August 28th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/197773762/">HBase Meetup @ Sift Science</a> in San Francisco</p>
-         <p>July 17th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/190994082/">HBase Meetup @ HP</a> in Sunnyvale</p>
-         <p>June 5th, 2014 <a href="http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/">HBase BOF at Hadoop Summit</a>, San Jose Convention Center</p>
-         <p>May 5th, 2014 <a href="http://www.hbasecon.com/">HBaseCon2014</a> at the Hilton San Francisco on Union Square</p>
-         <p>March 12th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/160757912/">HBase Meetup @ Ancestry.com</a> in San Francisco</p>
-      <p><small><a href="old_news.html">Old News</a></small></p>
-    </section>
-  </body>
-
-</document>

[14/27] hbase git commit: HBASE-18426 nightly job should use independent stages to check supported jdks

Posted by bu...@apache.org.
HBASE-18426 nightly job should use independent stages to check supported jdks

* Jenkinsfile that works for all current branches.
* adds dev-support script for setting parameters for our yetus nightly invocation
* quiets the "zip test results" step

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-18467
Commit: 04424ad65feae880d10fd3896e98a7201f76200b
Parents: 637f7ab
Author: Sean Busbey <bu...@apache.org>
Authored: Thu Jul 20 01:01:59 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Sun Aug 6 11:59:45 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile            | 256 +++++++++++++++++++-------------
 dev-support/hbase_nightly_yetus.sh |  86 +++++++++++
 2 files changed, 240 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/04424ad6/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 26f72d7..b5c7660 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -20,10 +20,6 @@ pipeline {
       label 'Hadoop'
     }
   }
-  // work around for YETUS-508, requires maven outside of the dockerfile
-  tools {
-    maven 'Maven (latest)'
-  }
   triggers {
     cron('@daily')
   }
@@ -38,8 +34,23 @@ pipeline {
     BASEDIR = "${env.WORKSPACE}/component"
     YETUS_RELEASE = '0.5.0'
     // where we'll write everything from different steps.
-    OUTPUT_RELATIVE = 'output'
-    OUTPUTDIR = "${env.WORKSPACE}/output"
+    OUTPUT_RELATIVE_GENERAL = 'output-general'
+    OUTPUTDIR_GENERAL = "${env.WORKSPACE}/output-general"
+    OUTPUT_RELATIVE_JDK7 = 'output-jdk7'
+    OUTPUTDIR_JDK7 = "${env.WORKSPACE}/output-jdk7"
+    OUTPUT_RELATIVE_JDK8 = 'output-jdk8'
+    OUTPUTDIR_JDK8 = "${env.WORKSPACE}/output-jdk8"
+    PROJECT = 'hbase'
+    PROJET_PERSONALITY = 'https://raw.githubusercontent.com/apache/hbase/master/dev-support/hbase-personality.sh'
+    // This section of the docs tells folks not to use the javadoc tag. older branches have our old version of the check for said tag.
+    AUTHOR_IGNORE_LIST = 'src/main/asciidoc/_chapters/developer.adoc,dev-support/test-patch.sh'
+    WHITESPACE_IGNORE_LIST = '.*/generated/.*'
+    // output from surefire; sadly the archive function in yetus only works on file names.
+    ARCHIVE_PATTERN_LIST = 'TEST-*.xml,org.apache.h*.txt'
+    // These tests currently have known failures. Once they burn down to 0, remove from here so that new problems will cause a failure.
+    TESTS_FILTER = 'cc,checkstyle,javac,javadoc,pylint,shellcheck,whitespace,perlcritic,ruby-lint,rubocop'
+    BRANCH_SPECIFIC_DOCKERFILE = "${env.BASEDIR}/dev-support/docker/Dockerfile"
+    EXCLUDE_TESTS_URL = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/excludes/'
   }
   parameters {
     booleanParam(name: 'USE_YETUS_PRERELEASE', defaultValue: false, description: '''Check to use the current HEAD of apache/yetus rather than our configured release.
@@ -48,24 +59,9 @@ pipeline {
     booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
   }
   stages {
-    stage ('yetus check') {
-      environment {
-        PROJECT = 'hbase'
-        PROJECT_PERSONALITY = 'https://git-wip-us.apache.org/repos/asf?p=hbase.git;a=blob_plain;f=dev-support/hbase-personality.sh;hb=refs/heads/master'
-        // This section of the docs tells folks not to use the javadoc tag. older branches have our old version of the check for said tag.
-        AUTHOR_IGNORE_LIST = 'src/main/asciidoc/_chapters/developer.adoc,dev-support/test-patch.sh'
-        WHITESPACE_IGNORE_LIST = '.*/generated/.*'
-        // output from surefire; sadly the archive function in yetus only works on file names.
-        ARCHIVE_PATTERN_LIST = 'TEST-*.xml,org.apache.h*-output.txt,org.apache.h*.txt'
-// These tests currently have known failures. Once they burn down to 0, remove from here so that new problems will cause a failure.
-        TESTS_FILTER = 'cc,checkstyle,javac,javadoc,pylint,shellcheck,whitespace,perlcritic,ruby-lint,rubocop'
-        BRANCH_SPECIFIC_DOCKERFILE = "${env.BASEDIR}/dev-support/docker/Dockerfile"
-        EXCLUDE_TESTS_URL = 'https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/excludes/'
-      }
+    stage ('yetus install') {
       steps {
-    // TODO we can move the yetus install into a different stage and then use stash to deploy it.
         sh  '''#!/usr/bin/env bash
-printenv
 echo "Ensure we have a copy of Apache Yetus."
 if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
   YETUS_DIR="${WORKSPACE}/yetus-${YETUS_RELEASE}"
@@ -101,96 +97,152 @@ if [ ! -d "${YETUS_DIR}" ]; then
   gunzip -c yetus.tar.gz | tar xpf - -C "${YETUS_DIR}" --strip-components 1
 fi
         '''
-    // TODO we can move the personality install into a different stage and then use stash to deploy it.
         dir ("${env.TOOLS}") {
           sh """#!/usr/bin/env bash
 echo "Downloading Project personality."
-curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
+curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
           """
         }
-// TODO break this out into a script so we can run shellcheck on it.
-        sh '''#!/usr/bin/env bash
-YETUS_ARGS=()
-YETUS_ARGS=("--multijdktests=compile,findbugs,unit" "${YETUS_ARGS[@]}")
-# On branch-1* this should point to jdk8, since the default JAVA_HOME will be jdk7.
-# On branch-2* this should be skipped, since we dropped jdk7 and JAVA_HOME will be jdk8
-# On master this should be skipped, since JAVA_HOME will be jdk8
-#YETUS_ARGS=("--multijdkdirs=/usr/lib/jvm/java-8-oracle" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--personality=${TOOLS}/personality.sh" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--basedir=${BASEDIR}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--archive-list=${ARCHIVE_PATTERN_LIST}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--console-urls" "${YETUS_ARGS[@]}")
-# YETUS-532, repeat this twice in case the fix is to update args rather than docs
-YETUS_ARGS=("--build-url-patchdir=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--build-url-artifacts=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--docker" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--dockerfile=${BRANCH_SPECIFIC_DOCKERFILE}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--empty-patch" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--html-report-file=${OUTPUTDIR}/console-report.html" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--jenkins" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--mvn-custom-repos" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--patch-dir=${OUTPUTDIR}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--project=${PROJECT}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--resetrepo" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--author-ignore-list=${AUTHOR_IGNORE_LIST}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--whitespace-eol-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
-
-if [[ true == "${DEBUG}" ]]; then
-  YETUS_ARGS=("--debug" "${YETUS_ARGS[@]}")
-fi
-
-rm -rf "${OUTPUTDIR}"
-mkdir -p "${OUTPUTDIR}"
-if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
-  YETUS_ARGS=("--shelldocs=${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/shelldocs" "${YETUS_ARGS[@]}")
-  TESTPATCHBIN="${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/test-patch"
-else
-  YETUS_ARGS=("--shelldocs=${WORKSPACE}/yetus-git/shelldocs/shelldocs.py" "${YETUS_ARGS[@]}")
-  TESTPATCHBIN="${WORKSPACE}/yetus-git/precommit/test-patch.sh"
-fi
-echo "Launching yetus with command line:"
-echo "${TESTPATCHBIN} ${YETUS_ARGS[*]}"
-
-/usr/bin/env bash "${TESTPATCHBIN}" "${YETUS_ARGS[@]}"
-        '''
+        stash name: 'yetus', includes: "yetus-*/*,yetus-*/**/*,tools/personality.sh"
       }
     }
-  }
-  post {
-    always {
-      // TODO confirm junit step accounts for multijdk results
-      junit 'output/**/target/**/TEST-*.xml'
-      // gzip surefire reports.
-      sh '''#!/bin/bash -e
-        if [ -d "${OUTPUTDIR}/archiver" ]; then
-          count=$(find "${OUTPUTDIR}/archiver" -type f | wc -l)
-          if [[ 0 -ne ${count} ]]; then
-            echo "zipping ${count} archived files"
-            zip -m -r "${OUTPUTDIR}/test_logs.zip" "${OUTPUTDIR}/archiver"
-          else
-            echo "No archived files, skipping compressing."
+    stage ('yetus general check') {
+      environment {
+        // TODO does hadoopcheck need to be jdk specific?
+        // Should be things that work with multijdk
+        TESTS = 'all,-unit,-findbugs'
+        // on branches that don't support jdk7, this will already be JAVA_HOME, so we'll end up not
+        // doing multijdk there.
+        MULTIJDK = '/usr/lib/jvm/java-8-oracle'
+        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_GENERAL}"
+        OUTPUTDIR = "${env.OUTPUTDIR_GENERAL}"
+      }
+      steps {
+        unstash 'yetus'
+        // TODO should this be a download from master, similar to how the personality is?
+        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+      }
+      post {
+        always {
+          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
+          archive 'output-general/*'
+          archive 'output-general/**/*'
+          publishHTML target: [
+            allowMissing: true,
+            keepAll: true,
+            alwaysLinkToLastBuild: true,
+            // has to be relative to WORKSPACE :(
+            reportDir: 'output-general',
+            reportFiles: 'console-report.html',
+            reportName: 'General Nightly Build Report'
+          ]
+        }
+      }
+    }
+    stage ('yetus jdk7 checks') {
+      when {
+        branch 'branch-1*'
+      }
+      environment {
+        TESTS = 'mvninstall,compile,javac,unit,htmlout'
+        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_JDK7}"
+        OUTPUTDIR = "${env.OUTPUTDIR_JDK7}"
+        // On branches where we do jdk7 checks, jdk7 will be JAVA_HOME already.
+      }
+      steps {
+        unstash 'yetus'
+        sh """#!/usr/bin/env bash
+          # for branch-1.1 we don't do jdk8 findbugs, so do it here
+          if [ "${env.BRANCH_NAME}" == "branch-1.1" ]; then
+            TESTS+=",findbugs"
           fi
-        else
-          echo "No archiver directory, skipping compressing."
-        fi
+          "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+        """
+      }
+      post {
+        always {
+          junit 'output-jdk7/**/target/**/TEST-*.xml'
+          // zip surefire reports.
+          sh '''#!/bin/bash -e
+            if [ -d "${OUTPUTDIR}/archiver" ]; then
+              count=$(find "${OUTPUTDIR}/archiver" -type f | wc -l)
+              if [[ 0 -ne ${count} ]]; then
+                echo "zipping ${count} archived files"
+                zip -q -m -r "${OUTPUTDIR}/test_logs.zip" "${OUTPUTDIR}/archiver"
+              else
+                echo "No archived files, skipping compressing."
+              fi
+            else
+              echo "No archiver directory, skipping compressing."
+            fi
 '''
-      // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
-      archive 'output/*'
-      archive 'output/**/*'
-      publishHTML target: [
-        allowMissing: true,
-        keepAll: true,
-        alwaysLinkToLastBuild: true,
-        // has to be relative to WORKSPACE :(
-        reportDir: 'output',
-        reportFiles: 'console-report.html',
-        reportName: 'Nightly Build Report'
-      ]
+          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
+          archive 'output-jdk7/*'
+          archive 'output-jdk7/**/*'
+          publishHTML target: [
+            allowMissing: true,
+            keepAll: true,
+            alwaysLinkToLastBuild: true,
+            // has to be relative to WORKSPACE :(
+            reportDir: 'output-jdk7',
+            reportFiles: 'console-report.html',
+            reportName: 'JDK7 Nightly Build Report'
+          ]
+        }
+      }
     }
+    stage ('yetus jdk8 checks') {
+      when {
+        not {
+          branch 'branch-1.1*'
+        }
+      }
+      environment {
+        TESTS = 'mvninstall,compile,javac,unit,findbugs,htmlout'
+        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_JDK8}"
+        OUTPUTDIR = "${env.OUTPUTDIR_JDK8}"
+        // This isn't strictly needed on branches that only support jdk8, but doesn't hurt
+        // and is needed on branches that do both jdk7 and jdk8
+        SET_JAVA_HOME = '/usr/lib/jvm/java-8-oracle'
+      }
+      steps {
+        unstash 'yetus'
+        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+      }
+      post {
+        always {
+          junit 'output-jdk8/**/target/**/TEST-*.xml'
+          // zip surefire reports.
+          sh '''#!/bin/bash -e
+            if [ -d "${OUTPUTDIR}/archiver" ]; then
+              count=$(find "${OUTPUTDIR}/archiver" -type f | wc -l)
+              if [[ 0 -ne ${count} ]]; then
+                echo "zipping ${count} archived files"
+                zip -q -m -r "${OUTPUTDIR}/test_logs.zip" "${OUTPUTDIR}/archiver"
+              else
+                echo "No archived files, skipping compressing."
+              fi
+            else
+              echo "No archiver directory, skipping compressing."
+            fi
+'''
+          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
+          archive 'output-jdk8/*'
+          archive 'output-jdk8/**/*'
+          publishHTML target: [
+            allowMissing: true,
+            keepAll: true,
+            alwaysLinkToLastBuild: true,
+            // has to be relative to WORKSPACE :(
+            reportDir: 'output-jdk8',
+            reportFiles: 'console-report.html',
+            reportName: 'JDK8 Nightly Build Report'
+          ]
+        }
+      }
+    }
+  }
+  post {
     failure {
       deleteDir()
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/04424ad6/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
new file mode 100755
index 0000000..007d64a
--- /dev/null
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -0,0 +1,86 @@
+#!/usr/bin/env bash
+# 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.
+
+declare -i missing_env=0
+# Validate params
+for required_env in "TESTS" "TOOLS" "BASEDIR" "ARCHIVE_PATTERN_LIST" "OUTPUT_RELATIVE" \
+                    "BRANCH_SPECIFIC_DOCKERFILE" "OUTPUTDIR" "PROJECT" "AUTHOR_IGNORE_LIST" \
+                    "WHITESPACE_IGNORE_LIST" "BRANCH_NAME" "TESTS_FILTER" "DEBUG" \
+                    "USE_YETUS_PRERELEASE" "WORKSPACE" "YETUS_RELEASE"; do
+  if [ -z "${!required_env}" ]; then
+    echo "[ERROR] Required environment variable '${required_env}' is not set."
+    missing_env=${missing_env}+1
+  fi
+done
+
+if [ ${missing_env} -gt 0 ]; then
+  echo "[ERROR] Please set the required environment variables before invoking. If this error is " \
+       "on Jenkins, then please file a JIRA about the error."
+  exit 1
+fi
+
+YETUS_ARGS=()
+if [[ -n "${MULTIJDK}" ]]; then
+  YETUS_ARGS=("--multijdktests=compile,javadoc" "${YETUS_ARGS[@]}")
+  YETUS_ARGS=("--multijdkdirs=${MULTIJDK}" "${YETUS_ARGS[@]}")
+fi
+
+if [[ -n "${SET_JAVA_HOME}" ]]; then
+  YETUS_ARGS=("--java-home=${SET_JAVA_HOME}" "${YETUS_ARGS[@]}")
+fi
+YETUS_ARGS=("--plugins=${TESTS}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--personality=${TOOLS}/personality.sh" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--basedir=${BASEDIR}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--archive-list=${ARCHIVE_PATTERN_LIST}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--console-urls" "${YETUS_ARGS[@]}")
+# YETUS-532, repeat this twice in case the fix is to update args rather than docs
+YETUS_ARGS=("--build-url-patchdir=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--build-url-artifacts=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--docker" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--dockerfile=${BRANCH_SPECIFIC_DOCKERFILE}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--empty-patch" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--html-report-file=${OUTPUTDIR}/console-report.html" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--jenkins" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--mvn-custom-repos" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--patch-dir=${OUTPUTDIR}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--project=${PROJECT}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--resetrepo" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--author-ignore-list=${AUTHOR_IGNORE_LIST}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--whitespace-eol-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--whitespace-tabs-ignore-list=${WHITESPACE_IGNORE_LIST}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--sentinel" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--branch=${BRANCH_NAME}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--tests-filter=${TESTS_FILTER}" "${YETUS_ARGS[@]}")
+
+if [[ true == "${DEBUG}" ]]; then
+  YETUS_ARGS=("--debug" "${YETUS_ARGS[@]}")
+fi
+
+rm -rf "${OUTPUTDIR}"
+mkdir -p "${OUTPUTDIR}"
+if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
+  YETUS_ARGS=("--shelldocs=${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/shelldocs" "${YETUS_ARGS[@]}")
+  TESTPATCHBIN="${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/test-patch"
+else
+  YETUS_ARGS=("--shelldocs=${WORKSPACE}/yetus-git/shelldocs/shelldocs.py" "${YETUS_ARGS[@]}")
+  TESTPATCHBIN="${WORKSPACE}/yetus-git/precommit/test-patch.sh"
+fi
+echo "Launching yetus with command line:"
+echo "${TESTPATCHBIN} ${YETUS_ARGS[*]}"
+
+/usr/bin/env bash "${TESTPATCHBIN}" "${YETUS_ARGS[@]}"

[08/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/index.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/index.adoc b/src/site/asciidoc/index.adoc
new file mode 100644
index 0000000..9b31c49
--- /dev/null
+++ b/src/site/asciidoc/index.adoc
@@ -0,0 +1,75 @@
+////
+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.
+////
+
+= Apache HBase&#153; Home
+
+.Welcome to Apache HBase(TM)
+link:http://www.apache.org/[Apache HBase(TM)] is the link:http://hadoop.apache.org[Hadoop] database, a distributed, scalable, big data store.
+
+.When Would I Use Apache HBase?
+Use Apache HBase when you need random, realtime read/write access to your Big Data. +
+This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware.
+
+Apache HBase is an open-source, distributed, versioned, non-relational database modeled after Google's link:http://research.google.com/archive/bigtable.html[Bigtable: A Distributed Storage System for Structured Data] by Chang et al.
+
+Just as Bigtable leverages the distributed data storage provided by the Google File System, Apache HBase provides Bigtable-like capabilities on top of Hadoop and HDFS.
+
+.Features
+- Linear and modular scalability.
+- Strictly consistent reads and writes.
+- Automatic and configurable sharding of tables
+- Automatic failover support between RegionServers.
+- Convenient base classes for backing Hadoop MapReduce jobs with Apache HBase tables.
+- Easy to use Java API for client access.
+- Block cache and Bloom Filters for real-time queries.
+- Query predicate push down via server side Filters
+- Thrift gateway and a REST-ful Web service that supports XML, Protobuf, and binary data encoding options
+- Extensible jruby-based (JIRB) shell
+- Support for exporting metrics via the Hadoop metrics subsystem to files or Ganglia; or via JMX
+
+.Where Can I Get More Information?
+See the link:book.html#arch.overview[Architecture Overview], the link:book.html#faq[FAQ] and the other documentation links at the top!
+
+.Export Control
+The HBase distribution includes cryptographic software. See the link:export_control.html[export control notice].
+
+== News
+Feb 17, 2015:: link:http://www.meetup.com/hbaseusergroup/events/219260093/[HBase meetup around Strata+Hadoop World] in San Jose
+
+January 15th, 2015:: link:http://www.meetup.com/hbaseusergroup/events/218744798/[HBase meetup @ AppDynamics] in San Francisco
+
+November 20th, 2014::  link:http://www.meetup.com/hbaseusergroup/events/205219992/[HBase meetup @ WANdisco] in San Ramon
+
+October 27th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/207386102/[HBase Meetup @ Apple] in Cupertino
+
+October 15th, 2014:: link:http://www.meetup.com/HBase-NYC/events/207655552[HBase Meetup @ Google] on the night before Strata/HW in NYC
+
+September 25th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/203173692/[HBase Meetup @ Continuuity] in Palo Alto
+
+August 28th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/197773762/[HBase Meetup @ Sift Science] in San Francisco
+
+July 17th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/190994082/[HBase Meetup @ HP] in Sunnyvale
+
+June 5th, 2014:: link:http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/[HBase BOF at Hadoop Summit], San Jose Convention Center
+
+May 5th, 2014:: link:http://www.hbasecon.com[HBaseCon2014] at the Hilton San Francisco on Union Square
+
+March 12th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/160757912/[HBase Meetup @ Ancestry.com] in San Francisco
+
+View link:old_news.html[Old News]

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/metrics.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/metrics.adoc b/src/site/asciidoc/metrics.adoc
new file mode 100644
index 0000000..be7d9a5
--- /dev/null
+++ b/src/site/asciidoc/metrics.adoc
@@ -0,0 +1,102 @@
+////
+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.
+////
+
+= Apache HBase (TM) Metrics
+
+== Introduction
+Apache HBase (TM) emits Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
+
+== Setup
+
+First read up on Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
+
+If you are using ganglia, the link:http://wiki.apache.org/hadoop/GangliaMetrics[GangliaMetrics] wiki page is useful read.
+
+To have HBase emit metrics, edit `$HBASE_HOME/conf/hadoop-metrics.properties` and enable metric 'contexts' per plugin.  As of this writing, hadoop supports *file* and *ganglia* plugins. Yes, the hbase metrics files is named hadoop-metrics rather than _hbase-metrics_ because currently at least the hadoop metrics system has the properties filename hardcoded. Per metrics _context_, comment out the NullContext and enable one or more plugins instead.
+
+If you enable the _hbase_ context, on regionservers you'll see total requests since last
+metric emission, count of regions and storefiles as well as a count of memstore size.
+On the master, you'll see a count of the cluster's requests.
+
+Enabling the _rpc_ context is good if you are interested in seeing
+metrics on each hbase rpc method invocation (counts and time taken).
+
+The _jvm_ context is useful for long-term stats on running hbase jvms -- memory used, thread counts, etc. As of this writing, if more than one jvm is running emitting metrics, at least in ganglia, the stats are aggregated rather than reported per instance.
+
+== Using with JMX
+
+In addition to the standard output contexts supported by the Hadoop 
+metrics package, you can also export HBase metrics via Java Management 
+Extensions (JMX).  This will allow viewing HBase stats in JConsole or 
+any other JMX client.
+
+=== Enable HBase stats collection
+
+To enable JMX support in HBase, first edit `$HBASE_HOME/conf/hadoop-metrics.properties` to support metrics refreshing. (If you've running 0.94.1 and above, or have already configured `hadoop-metrics.properties` for another output context, you can skip this step).
+[source,bash]
+----
+# Configuration of the "hbase" context for null
+hbase.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+hbase.period=60
+
+# Configuration of the "jvm" context for null
+jvm.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+jvm.period=60
+
+# Configuration of the "rpc" context for null
+rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
+rpc.period=60
+----
+
+=== Setup JMX Remote Access
+
+For remote access, you will need to configure JMX remote passwords and access profiles.  Create the files:
+`$HBASE_HOME/conf/jmxremote.passwd` (set permissions 
+        to 600):: +
+----
+monitorRole monitorpass
+controlRole controlpass
+----
+
+`$HBASE_HOME/conf/jmxremote.access`:: +
+----
+monitorRole readonly
+controlRole readwrite
+----
+
+=== Configure JMX in HBase startup
+
+Finally, edit the `$HBASE_HOME/conf/hbase-env.sh` script to add JMX support:
+[source,bash]
+----
+HBASE_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false"
+HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.password.file=$HBASE_HOME/conf/jmxremote.passwd"
+HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.access.file=$HBASE_HOME/conf/jmxremote.access"
+
+export HBASE_MASTER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10101"
+export HBASE_REGIONSERVER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10102"
+----
+
+After restarting the processes you want to monitor, you should now be able to run JConsole (included with the JDK since JDK 5.0) to view the statistics via JMX.  HBase MBeans are exported under the *`hadoop`* domain in JMX.
+
+
+== Understanding HBase Metrics
+
+For more information on understanding HBase metrics, see the link:book.html#hbase_metrics[metrics section] in the Apache HBase Reference Guide. 
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/old_news.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/old_news.adoc b/src/site/asciidoc/old_news.adoc
new file mode 100644
index 0000000..ae44caa
--- /dev/null
+++ b/src/site/asciidoc/old_news.adoc
@@ -0,0 +1,121 @@
+////
+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.
+////
+
+= Old Apache HBase (TM) News
+
+February 10th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/163139322/[HBase Meetup @ Continuuity] in Palo Alto
+
+January 30th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/158491762/[HBase Meetup @ Apple] in Cupertino
+
+January 30th, 2014:: link:http://www.meetup.com/Los-Angeles-HBase-User-group/events/160560282/[Los Angeles HBase User Group] in El Segundo
+
+October 24th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/140759692/[HBase User] and link:http://www.meetup.com/hackathon/events/144366512/[Developer] Meetup at HortonWorksin Palo Alto
+
+September 26, 2013:: link:http://www.meetup.com/hbaseusergroup/events/135862292/[HBase Meetup at Arista Networks] in San Francisco
+
+August 20th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/120534362/[HBase Meetup at Flurry] in San Francisco
+
+July 16th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/119929152/[HBase Meetup at Twitter] in San Francisco
+
+June 25th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/119154442/[Hadoop Summit Meetup].at San Jose Convention Center
+
+June 14th, 2013:: link:http://kijicon.eventbrite.com/[KijiCon: Building Big Data Apps] in San Francisco.
+
+June 13th, 2013:: link:http://www.hbasecon.com/[HBaseCon2013] in San Francisco.  Submit an Abstract!
+
+June 12th, 2013:: link:http://www.meetup.com/hackathon/events/123403802/[HBaseConHackAthon] at the Cloudera office in San Francisco.
+
+April 11th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/103587852/[HBase Meetup at AdRoll] in San Francisco
+
+February 28th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/96584102/[HBase Meetup at Intel Mission Campus]
+
+February 19th, 2013:: link:http://www.meetup.com/hackathon/events/103633042/[Developers PowWow] at HortonWorks' new digs
+
+January 23rd, 2013:: link:http://www.meetup.com/hbaseusergroup/events/91381312/[HBase Meetup at WibiData World HQ!]
+
+December 4th, 2012:: link:http://www.meetup.com/hackathon/events/90536432/[0.96 Bug Squashing and Testing Hackathon] at Cloudera, SF.
+
+October 29th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/82791572/[HBase User Group Meetup] at Wize Commerce in San Mateo.
+
+October 25th, 2012:: link:http://www.meetup.com/HBase-NYC/events/81728932/[Strata/Hadoop World HBase Meetup.] in NYC
+
+September 11th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/80621872/[Contributor's Pow-Wow at HortonWorks HQ.]
+
+August 8th, 2012:: link:http://www.apache.org/dyn/closer.cgi/hbase/[Apache HBase 0.94.1 is available for download]
+
+June 15th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/59829652/[Birds-of-a-feather] in San Jose, day after:: link:http://hadoopsummit.org[Hadoop Summit]
+
+May 23rd, 2012:: link:http://www.meetup.com/hackathon/events/58953522/[HackConAthon] in Palo Alto
+
+May 22nd, 2012:: link:http://www.hbasecon.com[HBaseCon2012] in San Francisco
+
+March 27th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/56021562/[Meetup @ StumbleUpon] in San Francisco
+
+January 19th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/46702842/[Meetup @ EBay]
+
+January 23rd, 2012:: Apache HBase 0.92.0 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
+
+December 23rd, 2011:: Apache HBase 0.90.5 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
+
+November 29th, 2011:: link:http://www.meetup.com/hackathon/events/41025972/[Developer Pow-Wow in SF] at Salesforce HQ
+
+November 7th, 2011:: link:http://www.meetup.com/hbaseusergroup/events/35682812/[HBase Meetup in NYC (6PM)] at the AppNexus office
+
+August 22nd, 2011:: link:http://www.meetup.com/hbaseusergroup/events/28518471/[HBase Hackathon (11AM) and Meetup (6PM)] at FB in PA
+
+June 30th, 2011:: link:http://www.meetup.com/hbaseusergroup/events/20572251/[HBase Contributor Day], the day after the:: link:http://developer.yahoo.com/events/hadoopsummit2011/[Hadoop Summit] hosted by Y!
+
+June 8th, 2011:: link:http://berlinbuzzwords.de/wiki/hbase-workshop-and-hackathon[HBase Hackathon] in Berlin to coincide with:: link:http://berlinbuzzwords.de/[Berlin Buzzwords]
+
+May 19th, 2011: Apache HBase 0.90.3 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
+
+April 12th, 2011: Apache HBase 0.90.2 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
+
+March 21st, 2011:: link:http://www.meetup.com/hackathon/events/16770852/[HBase 0.92 Hackathon at StumbleUpon, SF]
+February 22nd, 2011:: link:http://www.meetup.com/hbaseusergroup/events/16492913/[HUG12: February HBase User Group at StumbleUpon SF]
+December 13th, 2010:: link:http://www.meetup.com/hackathon/calendar/15597555/[HBase Hackathon: Coprocessor Edition]
+November 19th, 2010:: link:http://huguk.org/[Hadoop HUG in London] is all about Apache HBase
+November 15-19th, 2010:: link:http://www.devoxx.com/display/Devoxx2K10/Home[Devoxx] features HBase Training and multiple HBase presentations
+
+October 12th, 2010:: HBase-related presentations by core contributors and users at:: link:http://www.cloudera.com/company/press-center/hadoop-world-nyc/[Hadoop World 2010]
+
+October 11th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/14606174/[HUG-NYC: HBase User Group NYC Edition] (Night before Hadoop World)
+June 30th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/13562846/[Apache HBase Contributor Workshop] (Day after Hadoop Summit)
+May 10th, 2010:: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project 
+
+April 19, 2010:: Signup for link:http://www.meetup.com/hbaseusergroup/calendar/12689490/[HBase User Group Meeting, HUG10] hosted by Trend Micro
+
+March 10th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/12689351/[HBase User Group Meeting, HUG9] hosted by Mozilla
+
+January 27th, 2010:: Sign up for the link:http://www.meetup.com/hbaseusergroup/calendar/12241393/[HBase User Group Meeting, HUG8], at StumbleUpon in SF
+
+September 8th, 2010:: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release.  Get it off the link:http://www.apache.org/dyn/closer.cgi/hbase/[Releases] page.
+
+November 2-6th, 2009:: link:http://dev.us.apachecon.com/c/acus2009/[ApacheCon] in Oakland. The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.
+
+October 2nd, 2009:: HBase at Hadoop World in NYC. A few of us will be talking on Practical HBase out east at link:http://www.cloudera.com/hadoop-world-nyc[Hadoop World: NYC].
+
+August 7th-9th, 2009:: HUG7 and HBase Hackathon at StumbleUpon in SF: Sign up for the:: link:http://www.meetup.com/hbaseusergroup/calendar/10950511/[HBase User Group Meeting, HUG7] or for the link:http://www.meetup.com/hackathon/calendar/10951718/[Hackathon] or for both (all are welcome!).
+
+June, 2009::  HBase at HadoopSummit2009 and at NOSQL: See the link:http://wiki.apache.org/hadoop/HBase/HBasePresentations[presentations]
+
+March 3rd, 2009 :: HUG6 -- link:http://www.meetup.com/hbaseusergroup/calendar/9764004/[HBase User Group 6]
+
+January 30th, 2009:: LA Hbackathon: link:http://www.meetup.com/hbasela/calendar/9450876/[HBase January Hackathon Los Angeles] at link:http://streamy.com[Streamy] in Manhattan Beach
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/pseudo-distributed.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/pseudo-distributed.adoc b/src/site/asciidoc/pseudo-distributed.adoc
new file mode 100644
index 0000000..d13c63b
--- /dev/null
+++ b/src/site/asciidoc/pseudo-distributed.adoc
@@ -0,0 +1,23 @@
+////
+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.
+////
+
+
+= Running Apache HBase (TM) in pseudo-distributed mode
+This page has been retired.  The contents have been moved to the link:book.html#distributed[Distributed Operation: Pseudo- and Fully-distributed modes] section in the Reference Guide.
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/replication.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/replication.adoc b/src/site/asciidoc/replication.adoc
new file mode 100644
index 0000000..9089754
--- /dev/null
+++ b/src/site/asciidoc/replication.adoc
@@ -0,0 +1,22 @@
+////
+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.
+////
+
+= Apache HBase (TM) Replication
+
+This information has been moved to link:book.html#cluster_replication"[the Cluster Replication] section of the link:book.html[Apache HBase Reference Guide].

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/resources.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/resources.adoc b/src/site/asciidoc/resources.adoc
new file mode 100644
index 0000000..fef217e
--- /dev/null
+++ b/src/site/asciidoc/resources.adoc
@@ -0,0 +1,27 @@
+////
+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.
+////
+= Other Apache HBase (TM) Resources
+
+== Books
+HBase: The Definitive Guide:: link:http://shop.oreilly.com/product/0636920014348.do[HBase: The Definitive Guide, _Random Access to Your Planet-Size Data_] by Lars George. Publisher: O'Reilly Media, Released: August 2011, Pages: 556.
+
+HBase In Action:: link:http://www.manning.com/dimidukkhurana[HBase In Action] By Nick Dimiduk and Amandeep Khurana.  Publisher: Manning, MEAP Began: January 2012, Softbound print: Fall 2012, Pages: 350.
+
+HBase Administration Cookbook:: link:http://www.packtpub.com/hbase-administration-for-optimum-database-performance-cookbook/book[HBase Administration Cookbook] by Yifeng Jiang.  Publisher: PACKT Publishing, Release: Expected August 2012, Pages: 335.
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/sponsors.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/sponsors.adoc b/src/site/asciidoc/sponsors.adoc
new file mode 100644
index 0000000..4d7ebf3
--- /dev/null
+++ b/src/site/asciidoc/sponsors.adoc
@@ -0,0 +1,36 @@
+////
+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.
+////
+
+= Apache HBase(TM) Sponsors
+
+First off, thanks to link:http://www.apache.org/foundation/thanks.html[all who sponsor] our parent, the Apache Software Foundation.
+
+The below companies have been gracious enough to provide their commerical tool offerings free of charge to the Apache HBase(TM) project.
+
+* The crew at link:http://www.ej-technologies.com/[ej-technologies] have been letting us use link:http://www.ej-technologies.com/products/jprofiler/overview.html[JProfiler] for years now. 
+
+* The lads at link:http://headwaysoftware.com/[headway software] have given us a license for link:http://headwaysoftware.com/products/?code=Restructure101[Restructure101] so we can untangle our interdependency mess.
+
+* link:http://www.yourkit.com[YourKit] allows us to use their link:http://www.yourkit.com/overview/index.jsp[Java Profiler].
+* Some of us use link:http://www.jetbrains.com/idea[IntelliJ IDEA] thanks to link:http://www.jetbrains.com/[JetBrains].
+* Thank you to Boris at link:http://www.vectorportal.com/[Vector Portal] for granting us a license on the image on which our logo is based.
+
+== Sponsoring the Apache Software Foundation">
+To contribute to the Apache Software Foundation, a good idea in our opinion, see the link:http://www.apache.org/foundation/sponsorship.html[ASF Sponsorship] page.
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/custom/project-info-report.properties
----------------------------------------------------------------------
diff --git a/src/site/custom/project-info-report.properties b/src/site/custom/project-info-report.properties
new file mode 100644
index 0000000..912339e
--- /dev/null
+++ b/src/site/custom/project-info-report.properties
@@ -0,0 +1,303 @@
+# 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.
+
+report.cim.access                                                  = Access
+report.cim.anthill.intro                                           = Apache HBase&#8482; uses {Anthill, http://www.anthillpro.com/html/products/anthillos/}.
+report.cim.bamboo.intro                                            = Apache HBase&#8482; uses {Bamboo, http://www.atlassian.com/software/bamboo/}.
+report.cim.buildforge.intro                                        = Apache HBase&#8482; uses {Build Forge, http://www-306.ibm.com/software/awdtools/buildforge/enterprise/}.
+report.cim.continuum.intro                                         = Apache HBase&#8482; uses {Continuum, http://continuum.apache.org/}.
+report.cim.cruisecontrol.intro                                     = Apache HBase&#8482; uses {CruiseControl, http://cruisecontrol.sourceforge.net/}.
+report.cim.description                                             = These are the definitions of all continuous integration processes that builds and tests code on a frequent, regular basis.
+report.cim.general.intro                                           = Apache HBase&#8482; uses Continuous Integration System.
+report.cim.hudson.intro                                            = Apache HBase&#8482; uses {Hudson, http://hudson-ci.org/}.
+report.cim.jenkins.intro                                           = Apache HBase&#8482; uses {Jenkins, http://jenkins-ci.org/}.
+report.cim.luntbuild.intro                                         = Apache HBase&#8482; uses {Luntbuild, http://luntbuild.javaforge.com/}.
+report.cim.travis.intro                                            = Apache HBase&#8482; uses {Travis CI, https://travis-ci.org/}.
+report.cim.name                                                    = Continuous Integration
+report.cim.nocim                                                   = No continuous integration management system is defined. Please check back at a later date.
+report.cim.notifiers.column.address                                = Address
+report.cim.notifiers.column.configuration                          = Configuration
+report.cim.notifiers.column.type                                   = Type
+report.cim.notifiers.intro                                         = Configuration for notifying developers/users when a build is unsuccessful, including user information and notification mode.
+report.cim.notifiers.nolist                                        = No notifiers are defined. Please check back at a later date.
+report.cim.notifiers.title                                         = Notifiers
+report.cim.nourl                                                   = No url to the continuous integration system is defined.
+report.cim.overview.title                                          = Overview
+report.cim.title                                                   = Continuous Integration
+report.cim.url                                                     = This is a link to the continuous integration system used by the project:
+report.dependencies.column.artifactId                              = ArtifactId
+report.dependencies.column.classifier                              = Classifier
+report.dependencies.column.description                             = Description
+report.dependencies.column.groupId                                 = GroupId
+report.dependencies.column.license                                 = License
+report.dependencies.column.optional                                = Optional
+report.dependencies.column.isOptional                              = Yes
+report.dependencies.column.isNotOptional                           = No
+report.dependencies.column.type                                    = Type
+report.dependencies.column.url                                     = URL
+report.dependencies.column.version                                 = Version
+report.dependencies.description                                    = This is a list of project's dependencies and provides information on each dependency.
+report.dependencies.file.details.cell.debuginformation.yes         = Yes
+report.dependencies.file.details.cell.debuginformation.no          = No
+report.dependencies.file.details.column.classes                    = Classes
+report.dependencies.file.details.column.debuginformation           = Debug Information
+report.dependencies.file.details.column.entries                    = Entries
+report.dependencies.file.details.column.file                       = Filename
+report.dependencies.file.details.column.javaVersion                = Java Version
+report.dependencies.file.details.column.packages                   = Packages
+report.dependencies.file.details.column.sealed                     = Sealed
+report.dependencies.file.details.column.size                       = Size
+report.dependencies.file.details.column.size.gb                    = GB
+report.dependencies.file.details.column.size.mb                    = MB
+report.dependencies.file.details.column.size.kb                    = kB
+report.dependencies.file.details.columntitle.debuginformation      = Indicates whether these dependencies have been compiled with debug information.
+report.dependencies.file.details.title                             = Dependency File Details
+report.dependencies.file.details.total                             = Total
+report.dependencies.graph.tables.licenses                          = Licenses
+report.dependencies.graph.tables.unknown                           = Unknown
+report.dependencies.graph.title                                    = Apache HBase&#8482; Dependency Graph
+report.dependencies.graph.tree.title                               = Dependency Tree
+report.dependencies.intro.compile                                  = This is a list of compile dependencies for Apache HBase&#8482;. These dependencies are required to compile and run the application:
+report.dependencies.intro.provided                                 = This is a list of provided dependencies for Apache HBase&#8482;. These dependencies are required to compile the application, but should be provided by default when using the library:
+report.dependencies.intro.runtime                                  = This is a list of runtime dependencies for Apache HBase&#8482;. These dependencies are required to run the application:
+report.dependencies.intro.system                                   = This is a list of system dependencies for Apache HBase&#8482;. These dependencies are required to compile the application:
+report.dependencies.intro.test                                     = This is a list of test dependencies for Apache HBase&#8482;. These dependencies are only required to compile and run unit tests for the application:
+report.dependencies.name                                           = Dependencies
+report.dependencies.nolist                                         = There are no dependencies for Apache HBase&#8482;. It is a standalone application that does not depend on any other project.
+report.dependencies.repo.locations.artifact.breakdown              = Repository locations for each of the Dependencies.
+report.dependencies.repo.locations.cell.release.disabled           = No
+report.dependencies.repo.locations.cell.release.enabled            = Yes
+report.dependencies.repo.locations.cell.snapshot.disabled          = No
+report.dependencies.repo.locations.cell.snapshot.enabled           = Yes
+report.dependencies.repo.locations.cell.blacklisted.disabled       = No
+report.dependencies.repo.locations.cell.blacklisted.enabled        = Yes
+report.dependencies.repo.locations.column.artifact                 = Artifact
+report.dependencies.repo.locations.column.blacklisted              = Blacklisted
+report.dependencies.repo.locations.column.release                  = Release
+report.dependencies.repo.locations.column.repoid                   = Repo ID
+report.dependencies.repo.locations.column.snapshot                 = Snapshot
+report.dependencies.repo.locations.column.url                      = URL
+report.dependencies.repo.locations.title                           = Dependency Repository Locations
+report.dependencies.title                                          = Apache HBase&#8482; Dependencies
+report.dependencies.unnamed                                        = Unnamed
+report.dependencies.transitive.intro                               = This is a list of transitive dependencies for Apache HBase&#8482;. Transitive dependencies are the dependencies of the project dependencies.
+report.dependencies.transitive.nolist                              = No transitive dependencies are required for Apache HBase&#8482;.
+report.dependencies.transitive.title                               = Apache HBase&#8482; Transitive Dependencies
+report.dependency-convergence.convergence.caption                  = Dependencies used in modules
+report.dependency-convergence.convergence.single.caption           = Dependencies used in Apache HBase&#8482;
+report.dependency-convergence.description                          = This is the convergence of dependency versions across the entire project and its sub-modules.
+report.dependency-convergence.legend                               = Legend:
+report.dependency-convergence.legend.different                     = At least one dependency has a differing version of the dependency or has SNAPSHOT dependencies.
+report.dependency-convergence.legend.shared                        = All modules/dependencies share one version of the dependency.
+report.dependency-convergence.name                                 = Dependency Convergence
+report.dependency-convergence.reactor.name                         = Reactor Dependency Convergence
+report.dependency-convergence.reactor.title                        = Reactor Dependency Convergence
+report.dependency-convergence.stats.artifacts                      = Number of unique artifacts (NOA):
+report.dependency-convergence.stats.caption                        = Statistics:
+report.dependency-convergence.stats.convergence                    = Convergence (NOD/NOA):
+report.dependency-convergence.stats.dependencies                   = Number of dependencies (NOD):
+report.dependency-convergence.stats.readyrelease                   = Ready for release (100 % convergence and no SNAPSHOTS):
+report.dependency-convergence.stats.readyrelease.error             = Error
+report.dependency-convergence.stats.readyrelease.error.convergence = There is less than 100 % convergence.
+report.dependency-convergence.stats.readyrelease.error.snapshots   = There are SNAPSHOT dependencies.
+report.dependency-convergence.stats.readyrelease.success           = Success
+report.dependency-convergence.stats.conflicting                    = Number of version-conflicting artifacts (NOC):
+report.dependency-convergence.stats.snapshots                      = Number of SNAPSHOT artifacts (NOS):
+report.dependency-convergence.stats.modules                        = Number of modules:
+report.dependency-convergence.title                                = Dependency Convergence
+report.dependency-info.name                                        = Dependency Information
+report.dependency-info.title                                       = Dependency Information
+report.dependency-info.description                                 = These are instructions for including Apache HBase&#8482; as a dependency using various dependency management tools.
+report.index.nodescription                                         = There is currently no description associated with Apache HBase&#8482;.
+report.index.title                                                 = About Apache HBase&#8482;
+report.issuetracking.bugzilla.intro                                = Apache HBase&#8482; uses {Bugzilla, http://www.bugzilla.org/}.
+report.issuetracking.custom.intro                                  = Apache HBase&#8482; uses %issueManagementSystem% to manage its issues.
+report.issuetracking.description                                   = Apache HBase&#8482; uses the following issue management system(s).
+report.issuetracking.general.intro                                 = Apache HBase&#8482; uses an Issue Management System to manage its issues.
+report.issuetracking.intro                                         = Issues, bugs, and feature requests should be submitted to the following issue tracking system for Apache HBase&#8482;.
+report.issuetracking.jira.intro                                    = Apache HBase&#8482; uses {JIRA, http://www.atlassian.com/software/jira}.
+report.issuetracking.name                                          = Issue Tracking
+report.issuetracking.noissueManagement                             = No issue management system is defined. Please check back at a later date.
+report.issuetracking.overview.title                                = Overview
+report.issuetracking.scarab.intro                                  = Apache HBase&#8482; uses {Scarab, http://scarab.tigris.org/}.
+report.issuetracking.title                                         = Issue Tracking
+report.license.description                                         = Apache HBase&#8482; uses the following project license(s).
+report.license.multiple                                            = Apache HBase&#8482; is provided under multiple licenses:
+report.license.name                                                = Apache HBase&#8482; License
+report.license.nolicense                                           = No license is defined for Apache HBase&#8482;.
+report.license.overview.intro                                      = This is the license for the Apache HBase project itself, but not necessarily its dependencies.
+report.license.overview.title                                      = Overview
+report.license.originalText                                        = [Original text]
+report.license.copy                                                = Copy of the license follows:
+report.license.title                                               = Apache HBase&#8482; License
+report.license.unnamed                                             = Unnamed
+report.mailing-lists.column.archive                                = Archive
+report.mailing-lists.column.name                                   = Name
+report.mailing-lists.column.otherArchives                          = Other Archives
+report.mailing-lists.column.post                                   = Post
+report.mailing-lists.column.subscribe                              = Subscribe
+report.mailing-lists.column.unsubscribe                            = Unsubscribe
+report.mailing-lists.description                                   = These are Apache HBase&#8482;'s mailing lists.
+report.mailing-lists.intro                                         = For each list, links are provided to subscribe, unsubscribe, and view archives.
+report.mailing-lists.name                                          = Mailing Lists
+report.mailing-lists.nolist                                        = There are no mailing lists currently associated with Apache HBase&#8482;.
+report.mailing-lists.title                                         = Apache HBase&#8482; Mailing Lists
+report.scm.accessbehindfirewall.cvs.intro                          = If you are behind a firewall that blocks HTTP access to the CVS repository, you can use the {CVSGrab, http://cvsgrab.sourceforge.net/} web interface to checkout the source code.
+report.scm.accessbehindfirewall.general.intro                      = Refer to the documentation of the SCM used for more information about access behind a firewall.
+report.scm.accessbehindfirewall.svn.intro                          = If you are behind a firewall that blocks HTTP access to the Subversion repository, you can try to access it via the developer connection:
+report.scm.accessbehindfirewall.title                              = Access from Behind a Firewall
+report.scm.accessthroughtproxy.svn.intro1                          = The Subversion client can go through a proxy, if you configure it to do so. First, edit your "servers" configuration file to indicate which proxy to use. The file's location depends on your operating system. On Linux or Unix it is located in the directory "~/.subversion". On Windows it is in "%APPDATA%\\Subversion". (Try "echo %APPDATA%", note this is a hidden directory.)
+report.scm.accessthroughtproxy.svn.intro2                          = There are comments in the file explaining what to do. If you don't have that file, get the latest Subversion client and run any command; this will cause the configuration directory and template files to be created.
+report.scm.accessthroughtproxy.svn.intro3                          = Example: Edit the 'servers' file and add something like:
+report.scm.accessthroughtproxy.title                               = Access Through a Proxy
+report.scm.anonymousaccess.cvs.intro                               = Apache HBase&#8482;'s CVS repository can be checked out through anonymous CVS with the following instruction set. When prompted for a password for anonymous, simply press the Enter key.
+report.scm.anonymousaccess.general.intro                           = Refer to the documentation of the SCM used for more information about anonymously check out. The connection url is:
+report.scm.anonymousaccess.git.intro                               = The source can be checked out anonymously from Git with this command (See {http://git-scm.com/docs/git-clone,http://git-scm.com/docs/git-clone}):
+report.scm.anonymousaccess.hg.intro                                = The source can be checked out anonymously from Mercurial with this command (See {http://www.selenic.com/mercurial/hg.1.html#clone,http://www.selenic.com/mercurial/hg.1.html#clone}):
+report.scm.anonymousaccess.svn.intro                               = The source can be checked out anonymously from Subversion with this command:
+report.scm.anonymousaccess.title                                   = Anonymous Access
+report.scm.clearcase.intro                                         = Apache HBase&#8482; uses {ClearCase, http://www-306.ibm.com/software/awdtools/clearcase/} to manage its source code. Informations on ClearCase use can be found at {http://www.redbooks.ibm.com/redbooks/pdfs/sg246399.pdf, http://www.redbooks.ibm.com/redbooks/pdfs/sg246399.pdf}.
+report.scm.cvs.intro                                               = Apache HBase&#8482; uses {Concurrent Versions System, http://www.cvshome.org/} to manage its source code. Instructions on CVS use can be found at {http://cvsbook.red-bean.com/, http://cvsbook.red-bean.com/}.
+report.scm.description                                             = This document lists ways to access the online source repository.
+report.scm.devaccess.clearcase.intro                               = Only project developers can access the ClearCase tree via this method. Substitute username with the proper value.
+report.scm.devaccess.cvs.intro                                     = Only project developers can access the CVS tree via this method. Substitute username with the proper value.
+report.scm.devaccess.general.intro                                 = Refer to the documentation of the SCM used for more information about developer check out. The connection url is:
+report.scm.devaccess.git.intro                                     = Only project developers can access the Git tree via this method (See {http://git-scm.com/docs/git-clone,http://git-scm.com/docs/git-clone}).
+report.scm.devaccess.hg.intro                                      = Only project developers can access the Mercurial tree via this method (See {http://www.selenic.com/mercurial/hg.1.html#clone,http://www.selenic.com/mercurial/hg.1.html#clone}).
+report.scm.devaccess.perforce.intro                                = Only project developers can access the Perforce tree via this method. Substitute username and password with the proper values.
+report.scm.devaccess.starteam.intro                                = Only project developers can access the Starteam tree via this method. Substitute username with the proper value.
+report.scm.devaccess.svn.intro1.https                              = Everyone can access the Subversion repository via HTTP, but committers must checkout the Subversion repository via HTTPS.
+report.scm.devaccess.svn.intro1.other                              = Committers must checkout the Subversion repository.
+report.scm.devaccess.svn.intro1.svn                                = Committers must checkout the Subversion repository via SVN.
+report.scm.devaccess.svn.intro1.svnssh                             = Committers must checkout the Subversion repository via SVN+SSH.
+report.scm.devaccess.svn.intro2                                    = To commit changes to the repository, execute the following command to commit your changes (svn will prompt you for your password):
+report.scm.devaccess.title                                         = Developer Access
+report.scm.general.intro                                           = Apache HBase&#8482; uses a Source Content Management System to manage its source code.
+report.scm.name                                                    = Source Repository
+report.scm.noscm                                                   = No source configuration management system is defined. Please check back at a later date.
+report.scm.overview.title                                          = Overview
+report.scm.git.intro                                               = Apache HBase&#8482; uses {Git, http://git-scm.com/} to manage its source code. Instructions on Git use can be found at {http://git-scm.com/documentation,http://git-scm.com/documentation}.
+report.scm.hg.intro                                                = Apache HBase&#8482; uses {Mercurial, http://mercurial.selenic.com/wiki/} to manage its source code. Instructions on Mercurial use can be found at {http://hgbook.red-bean.com/read/, http://hgbook.red-bean.com/read/}.
+report.scm.perforce.intro                                          = Apache HBase&#8482; uses {Perforce, http://www.perforce.com/} to manage its source code. Instructions on Perforce use can be found at {http://www.perforce.com/perforce/doc.051/manuals/cmdref/index.html, http://www.perforce.com/perforce/doc.051/manuals/cmdref/index.html}.
+report.scm.starteam.intro                                          = Apache HBase&#8482; uses {Starteam, http://www.borland.com/us/products/starteam/} to manage its source code.
+report.scm.svn.intro                                               = Apache HBase&#8482; uses {Subversion, http://subversion.apache.org/} to manage its source code. Instructions on Subversion use can be found at {http://svnbook.red-bean.com/, http://svnbook.red-bean.com/}.
+report.scm.title                                                   = Source Repository
+report.scm.webaccess.nourl                                         = There is no browsable version of the source repository listed for Apache HBase&#8482;. Please check back again later.
+report.scm.webaccess.title                                         = Web Browser Access
+report.scm.webaccess.url                                           = The following is a link to a browsable version of the source repository:
+report.summary.build.artifactid                                    = ArtifactId
+report.summary.build.groupid                                       = GroupId
+report.summary.build.javaVersion                                   = Java Version
+report.summary.build.title                                         = Build Information
+report.summary.build.type                                          = Type
+report.summary.build.version                                       = Version
+report.summary.description                                         = This document lists other related information of Apache HBase&#8482;
+report.summary.field                                               = Field
+report.summary.general.description                                 = Description
+report.summary.general.homepage                                    = Homepage
+report.summary.general.name                                        = Name
+report.summary.general.title                                       = Project Information
+report.summary.name                                                = Project Summary
+report.summary.organization.name                                   = Name
+report.summary.organization.title                                  = Project Organization
+report.summary.organization.url                                    = URL
+report.summary.noorganization                                      = Apache HBase&#8482; does not belong to an organization.
+report.summary.title                                               = Project Summary
+report.summary.value                                               = Value
+report.summary.download                                            = Download
+report.team-list.contributors.actualtime                           = Actual Time (GMT)
+report.team-list.contributors.email                                = Email
+report.team-list.contributors.intro                                = The following additional people have contributed to Apache HBase&#8482; through the way of suggestions, patches or documentation.
+report.team-list.contributors.image                                = Image
+report.team-list.contributors.name                                 = Name
+report.team-list.contributors.organization                         = Organization
+report.team-list.contributors.organizationurl                      = Organization URL
+report.team-list.contributors.properties                           = Properties
+report.team-list.contributors.roles                                = Roles
+report.team-list.contributors.timezone                             = Time Zone
+report.team-list.contributors.title                                = Contributors
+report.team-list.contributors.url                                  = URL
+report.team-list.description                                       = These are the members of the Apache HBase&#8482; project. These are the individuals who have contributed to the project in one form or another.
+report.team-list.developers.actualtime                             = Actual Time (GMT)
+report.team-list.developers.email                                  = Email
+report.team-list.developers.image                                  = Image
+report.team-list.developers.id                                     = Id
+report.team-list.developers.intro                                  = These are the developers with commit privileges that have directly contributed to the project in one way or another.
+report.team-list.developers.name                                   = Name
+report.team-list.developers.organization                           = Organization
+report.team-list.developers.organizationurl                        = Organization URL
+report.team-list.developers.properties                             = Properties
+report.team-list.developers.roles                                  = Roles
+report.team-list.developers.timezone                               = Time Zone
+report.team-list.developers.title                                  = Members
+report.team-list.developers.url                                    = URL
+report.team-list.intro.description1                                = A successful project requires many people to play many roles. Some members write code or documentation, while others are valuable as testers, submitting patches and suggestions.
+report.team-list.intro.description2                                = The team is comprised of Members and Contributors. Members have direct access to the source of a project and actively evolve the code-base. Contributors improve the project through submission of patches and suggestions to the Members. The number of Contributors to the project is unbounded. Get involved today. All contributions to the project are greatly appreciated.
+report.team-list.intro.title                                       = The Team
+report.team-list.name                                              = Project Team
+report.team-list.nocontributor                                     = Apache HBase&#8482; does not maintain a list of contributors.
+report.team-list.nodeveloper                                       = Apache HBase&#8482; does not maintain a list of developers.
+report.team-list.title                                             = Project Team
+report.dependencyManagement.name                                   = Dependency Management
+report.dependencyManagement.description                            = This document lists the dependencies that are defined through dependencyManagement.
+report.dependencyManagement.title                                  = Project Dependency Management
+report.dependencyManagement.nolist                                 = There are no dependencies in the DependencyManagement of Apache HBase&#8482;.
+report.dependencyManagement.column.groupId                         = GroupId
+report.dependencyManagement.column.artifactId                      = ArtifactId
+report.dependencyManagement.column.version                         = Version
+report.dependencyManagement.column.classifier                      = Classifier
+report.dependencyManagement.column.type                            = Type
+report.dependencyManagement.column.license                         = License
+report.dependencyManagement.intro.compile                          = The following is a list of compile dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile and run the submodule:
+report.dependencyManagement.intro.provided                         = The following is a list of provided dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile the submodule, but should be provided by default when using the library:
+report.dependencyManagement.intro.runtime                          = The following is a list of runtime dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to run the submodule:
+report.dependencyManagement.intro.system                           = The following is a list of system dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile the submodule:
+report.dependencyManagement.intro.test                             = The following is a list of test dependencies in the DependencyManagement of Apache HBase&#8482;. These dependencies can be included in the submodules to compile and run unit tests for the submodule:
+report.pluginManagement.nolist                                     = There are no plugins defined in the PluginManagement part of Apache HBase&#8482;.
+report.pluginManagement.name                                       = Plugin Management
+report.pluginManagement.description                                = This document lists the plugins that are defined through pluginManagement.
+report.pluginManagement.title                                      = Project Plugin Management
+report.plugins.name                                                = Project Plugins
+report.plugins.description                                         = This document lists the build plugins and the report plugins used by Apache HBase&#8482;.
+report.plugins.title                                               = Project Build Plugins
+report.plugins.report.title                                        = Project Report Plugins
+report.plugins.nolist                                              = There are no plugins defined in the Build part of Apache HBase&#8482;.
+report.plugins.report.nolist                                       = There are no plugins reports defined in the Reporting part of Apache HBase&#8482;.
+report.modules.nolist                                              = There are no modules declared in Apache HBase&#8482;.
+report.modules.name                                                = Project Modules
+report.modules.description                                         = This document lists the modules (sub-projects) of Apache HBase&#8482;.
+report.modules.title                                               = Project Modules
+report.modules.intro                                               = Apache HBase&#8482; has declared the following modules:
+report.modules.header.name                                         = Name
+report.modules.header.description                                  = Description
+report.distributionManagement.name                                 = Distribution Management
+report.distributionManagement.description                          = This document provides informations on the distribution management of Apache HBase&#8482;.
+report.distributionManagement.title                                = Project Distribution Management
+report.distributionManagement.nodistributionmanagement             = No distribution management is defined for Apache HBase&#8482;.
+report.distributionManagement.overview.title                       = Overview
+report.distributionManagement.overview.intro                       = The following is the distribution management information used by Apache HBase&#8482;.
+report.distributionManagement.downloadURL                          = Download URL
+report.distributionManagement.repository                           = Repository
+report.distributionManagement.snapshotRepository                   = Snapshot Repository
+report.distributionManagement.site                                 = Site
+report.distributionManagement.relocation                           = Relocation
+report.distributionManagement.field                                = Field
+report.distributionManagement.value                                = Value
+report.distributionManagement.relocation.groupid                   = GroupId
+report.distributionManagement.relocation.artifactid                = ArtifactId
+report.distributionManagement.relocation.version                   = Version
+report.distributionManagement.relocation.message                   = Message

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/.htaccess
----------------------------------------------------------------------
diff --git a/src/site/resources/.htaccess b/src/site/resources/.htaccess
new file mode 100644
index 0000000..20bf651
--- /dev/null
+++ b/src/site/resources/.htaccess
@@ -0,0 +1,8 @@
+
+# Redirect replication URL to the right section of the book
+# Rule added 2015-1-12 -- can be removed in 6 months
+Redirect permanent /replication.html /book.html#_cluster_replication
+
+# Redirect old page-per-chapter book sections to new single file.
+RedirectMatch permanent ^/book/(.*)\.html$ /book.html#$1
+RedirectMatch permanent ^/book/$ /book.html

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/book/.empty
----------------------------------------------------------------------
diff --git a/src/site/resources/book/.empty b/src/site/resources/book/.empty
new file mode 100644
index 0000000..5513814
--- /dev/null
+++ b/src/site/resources/book/.empty
@@ -0,0 +1 @@
+# This directory is here so that we can have rewrite rules in our .htaccess to maintain old links. Otherwise we fall under some top-level niceness redirects because we have a file named book.html.

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/css/site.css
----------------------------------------------------------------------
diff --git a/src/site/resources/css/site.css b/src/site/resources/css/site.css
new file mode 100644
index 0000000..3f42f5a
--- /dev/null
+++ b/src/site/resources/css/site.css
@@ -0,0 +1,118 @@
+/*
+ * 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.
+ */
+
+/*@import(https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap.min.css);
+@import(https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap-responsive.css);*/
+html {
+  background-color: #fff;
+}
+body {
+  font-size: 16px;
+}
+li {
+  line-height: 120%;
+}
+
+div#topbar,
+div#banner,
+div#breadcrumbs,
+div#bodyColumn,
+footer {
+  width: initial;
+  padding-left: 20px;
+  padding-right: 20px;
+  clear: both;
+}
+footer {
+  background-color: #e5e5e5;
+}
+footer .row, footer p, footer .pull-right {
+  margin: 5px;
+}
+div#search-form.navbar-search.pull-right {
+  width: 290px;
+  margin-right: 0;
+  margin-top: -5px;
+  margin-left: 0;
+  position: initial;
+}
+li#publishDate.pull-right {
+  list-style: none;
+}
+.container,
+.navbar-static-top .container,
+.navbar-fixed-top .container,
+.navbar-fixed-bottom .container,
+.navbar-inner {
+  width: initial;
+}
+/* Change the color and effect when clicking in menus */
+.dropdown-menu>li>a:hover,
+.dropdown-menu>li>a:focus,
+.dropdown-submenu:hover>a,
+.dropdown-submenu:focus>a {
+  background-color: #e5e5e5;
+  background-image: none;
+  color: #000;
+  font-weight: bolder;
+}
+
+.dropdown-backdrop {
+  position: static;
+}
+
+@media only screen and (max-width: 979px) {
+  body {
+    padding-left: 0;
+    padding-right: 0;
+    width: initial;
+    margin: 0;
+  }
+  /* Without this rule, drop-down divs are a fixed height
+   * the first time they are expanded */
+  .collapse.in {
+      height: auto !important;
+  }
+  div#search-form.navbar-search.pull-right {
+    padding: 0;
+    margin-left: ;
+    width: initial;
+    clear: both;
+  }
+}
+
+/* Fix Google Custom Search results on very narrow screens */
+@media(max-width: 480px) {
+    .gsc-overflow-hidden .nav-collapse {
+        -webkit-transform: none;
+    }
+}
+
+/* Override weird body padding thing that causes scrolling */
+@media (max-width: 767px)
+body {
+    padding-right: 0;
+    padding-left: 0;
+}
+
+@media (max-width: 767px)
+.navbar-fixed-top, .navbar-fixed-bottom, .navbar-static-top {
+  margin-left: 0;
+  margin-right: 0;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/doap_Hbase.rdf
----------------------------------------------------------------------
diff --git a/src/site/resources/doap_Hbase.rdf b/src/site/resources/doap_Hbase.rdf
new file mode 100644
index 0000000..46082a1
--- /dev/null
+++ b/src/site/resources/doap_Hbase.rdf
@@ -0,0 +1,57 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl"?>
+<rdf:RDF xml:lang="en"
+         xmlns="http://usefulinc.com/ns/doap#" 
+         xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#" 
+         xmlns:asfext="http://projects.apache.org/ns/asfext#"
+         xmlns:foaf="http://xmlns.com/foaf/0.1/">
+<!--
+    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.
+-->
+  <Project rdf:about="http://hbase.apache.org">
+    <created>2012-04-14</created>
+    <license rdf:resource="http://usefulinc.com/doap/licenses/asl20" />
+    <name>Apache HBase</name>
+    <homepage rdf:resource="http://hbase.apache.org" />
+    <asfext:pmc rdf:resource="http://hbase.apache.org" />
+    <shortdesc>Apache HBase software is the Hadoop database. Think of it as a distributed, scalable, big data store.</shortdesc>
+    <description>Use Apache HBase software when you need random, realtime read/write access to your Big Data. This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware. HBase is an open-source, distributed, versioned, column-oriented store modeled after Google's Bigtable: A Distributed Storage System for Structured Data by Chang et al. Just as Bigtable leverages the distributed data storage provided by the Google File System, HBase provides Bigtable-like capabilities on top of Hadoop and HDFS. </description>
+    <bug-database rdf:resource="http://issues.apache.org/jira/browse/HBASE" />
+    <mailing-list rdf:resource="http://hbase.apache.org/mail-lists.html" />
+    <download-page rdf:resource="http://www.apache.org/dyn/closer.cgi/hbase/" />
+    <programming-language>Java</programming-language>
+    <category rdf:resource="http://projects.apache.org/category/database" />
+    <release>
+      <Version>
+        <name>Apache hbase </name>
+        <created>2015-07-23</created>
+        <revision>2.0.0-SNAPSHOT</revision>
+      </Version>
+    </release>
+    <repository>
+      <GitRepository>
+        <location rdf:resource="git://git.apache.org/hbase.git"/>
+        <browse rdf:resource="https://git-wip-us.apache.org/repos/asf?p=hbase.git"/>
+      </GitRepository>
+    </repository>
+    <maintainer>
+      <foaf:Person>
+        <foaf:name>Apache HBase PMC</foaf:name>
+          <foaf:mbox rdf:resource="mailto:dev@hbase.apache.org"/>
+      </foaf:Person>
+    </maintainer>
+  </Project>
+</rdf:RDF>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/architecture.gif
----------------------------------------------------------------------
diff --git a/src/site/resources/images/architecture.gif b/src/site/resources/images/architecture.gif
new file mode 100644
index 0000000..8d84a23
Binary files /dev/null and b/src/site/resources/images/architecture.gif differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/bc_basic.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bc_basic.png b/src/site/resources/images/bc_basic.png
new file mode 100644
index 0000000..231de93
Binary files /dev/null and b/src/site/resources/images/bc_basic.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/bc_config.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bc_config.png b/src/site/resources/images/bc_config.png
new file mode 100644
index 0000000..53250cf
Binary files /dev/null and b/src/site/resources/images/bc_config.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/bc_l1.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bc_l1.png b/src/site/resources/images/bc_l1.png
new file mode 100644
index 0000000..36d7e55
Binary files /dev/null and b/src/site/resources/images/bc_l1.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/bc_l2_buckets.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bc_l2_buckets.png b/src/site/resources/images/bc_l2_buckets.png
new file mode 100644
index 0000000..5163928
Binary files /dev/null and b/src/site/resources/images/bc_l2_buckets.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/bc_stats.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bc_stats.png b/src/site/resources/images/bc_stats.png
new file mode 100644
index 0000000..d8c6384
Binary files /dev/null and b/src/site/resources/images/bc_stats.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/big_h_logo.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/big_h_logo.png b/src/site/resources/images/big_h_logo.png
new file mode 100644
index 0000000..5256094
Binary files /dev/null and b/src/site/resources/images/big_h_logo.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/big_h_logo.svg
----------------------------------------------------------------------
diff --git a/src/site/resources/images/big_h_logo.svg b/src/site/resources/images/big_h_logo.svg
new file mode 100644
index 0000000..ab24198
--- /dev/null
+++ b/src/site/resources/images/big_h_logo.svg
@@ -0,0 +1,139 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Generator: Adobe Illustrator 15.1.0, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   version="1.1"
+   id="Layer_1"
+   x="0px"
+   y="0px"
+   width="792px"
+   height="612px"
+   viewBox="0 0 792 612"
+   enable-background="new 0 0 792 612"
+   xml:space="preserve"
+   inkscape:version="0.48.4 r9939"
+   sodipodi:docname="big_h_same_font_hbase3_logo.png"
+   inkscape:export-filename="big_h_bitmap.png"
+   inkscape:export-xdpi="90"
+   inkscape:export-ydpi="90"><metadata
+   id="metadata3693"><rdf:RDF><cc:Work
+       rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
+         rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
+   id="defs3691" /><sodipodi:namedview
+   pagecolor="#000000"
+   bordercolor="#666666"
+   borderopacity="1"
+   objecttolerance="10"
+   gridtolerance="10"
+   guidetolerance="10"
+   inkscape:pageopacity="0"
+   inkscape:pageshadow="2"
+   inkscape:window-width="1440"
+   inkscape:window-height="856"
+   id="namedview3689"
+   showgrid="false"
+   inkscape:zoom="2.1814013"
+   inkscape:cx="415.39305"
+   inkscape:cy="415.72702"
+   inkscape:window-x="1164"
+   inkscape:window-y="22"
+   inkscape:window-maximized="0"
+   inkscape:current-layer="Layer_1" />
+
+
+
+
+
+
+<text
+   xml:space="preserve"
+   style="font-size:40px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
+   x="311.18643"
+   y="86.224579"
+   id="text3082"
+   sodipodi:linespacing="125%"><tspan
+     sodipodi:role="line"
+     id="tspan3084"
+     x="311.18643"
+     y="86.224579" /></text>
+<text
+   xml:space="preserve"
+   style="font-size:40px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#000000;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
+   x="283.95764"
+   y="87.845337"
+   id="text3086"
+   sodipodi:linespacing="125%"><tspan
+     sodipodi:role="line"
+     id="tspan3088"
+     x="283.95764"
+     y="87.845337" /></text>
+<g
+   id="g3105"
+   transform="translate(14.669469,-80.682082)"
+   inkscape:export-filename="/Users/stack/Documents/big_h_base.png"
+   inkscape:export-xdpi="90"
+   inkscape:export-ydpi="90"><path
+     sodipodi:nodetypes="ccccccccccccccccccccccccccccc"
+     style="fill:#ba160c"
+     inkscape:connector-curvature="0"
+     id="path3677"
+     d="m 589.08202,499.77746 -40.3716,0 0,-168.36691 40.3716,0 z m -40.20304,-168.35619 -0.1684,-104.30857 40.3716,0 -0.33048,104.26805 z m -0.1684,168.35619 -40.37568,0 0,-104.82988 -259.42272,0 0,104.82988 -79.42128,0 0,-272.66476 79.42128,0 0,104.29785 224.92224,0 34.50456,0 40.37568,0 0,168.36691 z m 0,-272.66476 -40.37568,0 -0.0171,104.30857 40.55802,-0.01 z"
+     inkscape:export-filename="/Users/stack/Documents/polygon3687.png"
+     inkscape:export-xdpi="90"
+     inkscape:export-ydpi="90" /><path
+     sodipodi:nodetypes="cscsccsssccsssccscsccccccccccccccccccccc"
+     style="fill:#ba160c"
+     inkscape:connector-curvature="0"
+     id="path3679"
+     d="m 263.96692,553.27262 c 6.812,4.218 10.219,10.652 10.219,19.303 0,6.272 -2,11.571 -6.002,15.897 -4.325,4.758 -10.165,7.137 -17.519,7.137 l -28.629,0 0,-19.465 28.629,0 c 2.812,0 4.218,-2.109 4.218,-6.327 0,-4.216 -1.406,-6.325 -4.218,-6.325 l -28.629,0 0,-19.303 27.17,0 c 2.811,0 4.217,-2.109 4.217,-6.327 0,-4.216 -1.406,-6.326 -4.217,-6.326 l -27.17,0 0,-19.464 27.17,0 c 7.353,0 13.192,2.379 17.519,7.137 3.892,4.325 5.839,9.625 5.839,15.896 0,7.787 -2.866,13.842 -8.597,18.167 z m -41.931,42.338 -52.312,0 0,-51.42 19.466,0 5.259,0 27.588,0 0,19.303 -32.847,0 0,12.652 32.847,0 0,19.465 z m 0,-64.073 -32.847,0 0.0405,12.76351 -19.466,0.081 -0.0405,-32.30954 52.312,0 0,19.465 z" /><path
+     style="fill:#ba160c"
+     inkscape:connector-curvature="0"
+     id="path3683"
+     d="m 384.35292,595.61062 h -19.465 v -26.602 h -31.094 -0.618 v -19.466 h 0.618 31.094 v -11.68 c 0,-4.216 -1.406,-6.324 -4.218,-6.324 h -27.494 v -19.465 h 27.494 c 7.03,0 12.733,2.541 17.114,7.623 4.379,5.083 6.569,11.139 6.569,18.167 v 57.747 z m -51.177,-26.602 h -19.547 -12.165 v 26.602 h -19.466 v -57.748 c 0,-7.028 2.19,-13.083 6.569,-18.167 4.379,-5.083 10.03,-7.623 16.952,-7.623 h 27.656 v 19.466 h -27.656 c -2.704,0 -4.055,2.108 -4.055,6.324 v 11.68 h 12.165 19.547 v 19.466 z" /><path
+     style="fill:#ba160c"
+     inkscape:connector-curvature="0"
+     id="path3685"
+     d="m 492.35692,569.81862 c 0,7.03 -2.109,13.031 -6.327,18.006 -4.541,5.19 -10.273,7.786 -17.193,7.786 h -72.02 v -19.465 h 72.02 c 2.704,0 4.055,-2.109 4.055,-6.327 0,-4.216 -1.352,-6.325 -4.055,-6.325 h -52.394 c -6.92,0 -12.652,-2.596 -17.193,-7.787 -4.327,-4.865 -6.49,-10.813 -6.49,-17.843 0,-7.028 2.218,-13.083 6.651,-18.167 4.434,-5.083 10.112,-7.623 17.032,-7.623 h 72.021 v 19.464 h -72.021 c -2.703,0 -4.055,2.109 -4.055,6.326 0,4.109 1.352,6.164 4.055,6.164 h 52.394 c 6.92,0 12.652,2.596 17.193,7.787 4.218,4.974 6.327,10.976 6.327,18.004 z" /><polygon
+     style="fill:#ba160c"
+     transform="translate(-71.972085,223.93862)"
+     id="polygon3687"
+     points="656.952,339.555 591.906,339.555 591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 "
+     inkscape:export-xdpi="90"
+     inkscape:export-ydpi="90" /><g
+     id="g3349"><g
+       id="g3344"><text
+         transform="scale(0.93350678,1.0712295)"
+         sodipodi:linespacing="125%"
+         id="text3076"
+         y="203.03328"
+         x="181.98402"
+         style="font-size:84.015625px;font-style:italic;font-variant:normal;font-weight:bold;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:0px;word-spacing:0px;writing-mode:lr-tb;text-anchor:start;fill:#4d4d4d;fill-opacity:1;stroke:none;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi Bold Italic"
+         xml:space="preserve"
+         inkscape:export-xdpi="90"
+         inkscape:export-ydpi="90"
+         inkscape:export-filename="/Users/stack/Documents/polygon3687.png"><tspan
+           style="font-size:84.015625px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;letter-spacing:25.64349174px;writing-mode:lr-tb;text-anchor:start;fill:#4d4d4d;font-family:Bitsumishi;-inkscape-font-specification:Bitsumishi"
+           y="203.03328"
+           x="181.98402"
+           id="tspan3080"
+           sodipodi:role="line">APACHE</tspan></text>
+<rect
+         y="191.93103"
+         x="178.85117"
+         height="10.797735"
+         width="7.7796612"
+         id="rect3090"
+         style="fill:#4d4d4d" /></g><rect
+       style="fill:#4d4d4d"
+       id="rect3103"
+       width="8.1443329"
+       height="10.787481"
+       x="334.64697"
+       y="191.93881" /></g></g></svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/data_block_diff_encoding.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/data_block_diff_encoding.png b/src/site/resources/images/data_block_diff_encoding.png
new file mode 100644
index 0000000..0bd03a4
Binary files /dev/null and b/src/site/resources/images/data_block_diff_encoding.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/data_block_no_encoding.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/data_block_no_encoding.png b/src/site/resources/images/data_block_no_encoding.png
new file mode 100644
index 0000000..56498b4
Binary files /dev/null and b/src/site/resources/images/data_block_no_encoding.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/data_block_prefix_encoding.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/data_block_prefix_encoding.png b/src/site/resources/images/data_block_prefix_encoding.png
new file mode 100644
index 0000000..4271847
Binary files /dev/null and b/src/site/resources/images/data_block_prefix_encoding.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/favicon.ico
----------------------------------------------------------------------
diff --git a/src/site/resources/images/favicon.ico b/src/site/resources/images/favicon.ico
new file mode 100644
index 0000000..6e4d0f7
Binary files /dev/null and b/src/site/resources/images/favicon.ico differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hadoop-logo.jpg
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hadoop-logo.jpg b/src/site/resources/images/hadoop-logo.jpg
new file mode 100644
index 0000000..809525d
Binary files /dev/null and b/src/site/resources/images/hadoop-logo.jpg differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbase_logo.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbase_logo.png b/src/site/resources/images/hbase_logo.png
new file mode 100644
index 0000000..e962ce0
Binary files /dev/null and b/src/site/resources/images/hbase_logo.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbase_logo.svg
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbase_logo.svg b/src/site/resources/images/hbase_logo.svg
new file mode 100644
index 0000000..2cc26d9
--- /dev/null
+++ b/src/site/resources/images/hbase_logo.svg
@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Generator: Adobe Illustrator 15.1.0, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   version="1.1"
+   id="Layer_1"
+   x="0px"
+   y="0px"
+   width="792px"
+   height="612px"
+   viewBox="0 0 792 612"
+   enable-background="new 0 0 792 612"
+   xml:space="preserve"
+   inkscape:version="0.48.4 r9939"
+   sodipodi:docname="hbase_banner_logo.png"
+   inkscape:export-filename="hbase_logo_filledin.png"
+   inkscape:export-xdpi="90"
+   inkscape:export-ydpi="90"><metadata
+   id="metadata3285"><rdf:RDF><cc:Work
+       rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
+         rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
+   id="defs3283" /><sodipodi:namedview
+   pagecolor="#ffffff"
+   bordercolor="#666666"
+   borderopacity="1"
+   objecttolerance="10"
+   gridtolerance="10"
+   guidetolerance="10"
+   inkscape:pageopacity="0"
+   inkscape:pageshadow="2"
+   inkscape:window-width="1131"
+   inkscape:window-height="715"
+   id="namedview3281"
+   showgrid="false"
+   inkscape:zoom="4.3628026"
+   inkscape:cx="328.98554"
+   inkscape:cy="299.51695"
+   inkscape:window-x="752"
+   inkscape:window-y="456"
+   inkscape:window-maximized="0"
+   inkscape:current-layer="Layer_1" />
+<path
+   d="m 233.586,371.672 -9.895,0 0,-51.583 9.895,0 0,51.583 z m -9.77344,-51.59213 -0.12156,-31.94487 9.895,0 -0.0405,31.98539 z m -0.12156,51.59213 -9.896,0 0,-32.117 -63.584,0 0,32.117 -19.466,0 0,-83.537 19.466,0 0,31.954 55.128,0 8.457,0 9.896,0 0,51.583 z m 0,-83.537 -9.896,0 0,31.98539 10.01756,-0.0405 z"
+   id="path3269"
+   inkscape:connector-curvature="0"
+   style="fill:#ba160c"
+   sodipodi:nodetypes="cccccccccccccccccccccccccccccc" />
+<path
+   d="m 335.939,329.334 c 6.812,4.218 10.219,10.652 10.219,19.303 0,6.272 -2,11.571 -6.002,15.897 -4.325,4.758 -10.165,7.137 -17.519,7.137 l -28.629,0 0,-19.465 28.629,0 c 2.812,0 4.218,-2.109 4.218,-6.327 0,-4.216 -1.406,-6.325 -4.218,-6.325 l -28.629,0 0,-19.303 27.17,0 c 2.811,0 4.217,-2.109 4.217,-6.327 0,-4.216 -1.406,-6.326 -4.217,-6.326 l -27.17,0 0,-19.464 27.17,0 c 7.353,0 13.192,2.379 17.519,7.137 3.892,4.325 5.839,9.625 5.839,15.896 0,7.787 -2.866,13.842 -8.597,18.167 z m -41.931,42.338 -52.312,0 0,-51.42 19.466,0 5.259,0 27.588,0 0,19.303 -32.847,0 0,12.652 32.847,0 0,19.465 z m 0,-64.073 -32.847,0 0.0405,13.24974 -19.466,-0.48623 -0.0405,-32.22851 52.312,0 0,19.465 z"
+   id="path3271"
+   inkscape:connector-curvature="0"
+   style="fill:#ba160c"
+   sodipodi:nodetypes="cscsccsssccsssccscsccccccccccccccccccccc" />
+<path
+   d="M355.123,266.419v-8.92h14.532v-5.353c0-1.932-0.644-2.899-1.933-2.899h-12.6v-8.919h12.6  c3.223,0,5.836,1.164,7.842,3.494c2.007,2.33,3.011,5.104,3.011,8.325v26.463h-8.921v-12.19H355.123L355.123,266.419z   M473.726,278.61h-29.587c-3.469,0-6.417-1.152-8.845-3.458c-2.429-2.304-3.642-5.191-3.642-8.659v-14.049  c0-3.47,1.213-6.356,3.642-8.662c2.428-2.304,5.376-3.455,8.845-3.455h29.587v8.919h-29.587c-2.378,0-3.567,1.066-3.567,3.197  v14.049c0,2.131,1.189,3.196,3.567,3.196h29.587V278.61L473.726,278.61z M567.609,278.61h-8.996v-14.718h-22.895v14.718h-8.92  v-38.282h8.92v14.644h22.895v-14.644h8.996V278.61L567.609,278.61z M661.494,249.247h-31.889v5.725h29.807v8.92h-29.807v5.797  h31.814v8.92h-40.735v-38.282h40.809V249.247z M355.123,240.328v8.919h-12.674c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h2.435  h6.522v8.92h-6.522h-2.435h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.011-8.325c2.006-2.33,4.596-3.494,7.768-3.494H355.123  L355.123,240.328z M254.661,266.122v-8.92h13.083c1.288,0,1.
 933-1.313,1.933-3.939c0-2.676-0.645-4.015-1.933-4.015h-13.083v-8.919  h13.083c3.32,0,5.995,1.363,8.028,4.088c1.883,2.478,2.825,5.425,2.825,8.846c0,3.419-0.942,6.342-2.825,8.771  c-2.033,2.725-4.708,4.088-8.028,4.088H254.661z M177.649,278.61h-8.92v-12.19h-14.532v-8.92h14.532v-5.353  c0-1.932-0.644-2.899-1.932-2.899h-12.6v-8.919h12.6c3.222,0,5.835,1.164,7.842,3.494c2.007,2.33,3.01,5.104,3.01,8.325V278.61  L177.649,278.61z M254.661,240.328v8.919h-15.016v7.954h15.016v8.92h-15.016v12.488h-8.92v-38.282H254.661z M154.198,266.419h-7.604  h-1.354h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.01-8.325c2.007-2.33,4.597-3.494,7.768-3.494h12.674v8.919h-12.674  c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h1.354h7.604V266.419z"
+   id="path3273"
+   style="fill:#666666"
+   fill="#878888" />
+<path
+   fill="#BA160C"
+   d="M456.325,371.672H436.86V345.07h-31.094h-0.618v-19.466h0.618h31.094v-11.68  c0-4.216-1.406-6.324-4.218-6.324h-27.494v-19.465h27.494c7.03,0,12.733,2.541,17.114,7.623c4.379,5.083,6.569,11.139,6.569,18.167  V371.672z M405.148,345.07h-19.547h-12.165v26.602h-19.466v-57.748c0-7.028,2.19-13.083,6.569-18.167  c4.379-5.083,10.03-7.623,16.952-7.623h27.656V307.6h-27.656c-2.704,0-4.055,2.108-4.055,6.324v11.68h12.165h19.547V345.07z"
+   id="path3275" />
+<path
+   fill="#BA160C"
+   d="M564.329,345.88c0,7.03-2.109,13.031-6.327,18.006c-4.541,5.19-10.273,7.786-17.193,7.786h-72.02v-19.465  h72.02c2.704,0,4.055-2.109,4.055-6.327c0-4.216-1.352-6.325-4.055-6.325h-52.394c-6.92,0-12.652-2.596-17.193-7.787  c-4.327-4.865-6.49-10.813-6.49-17.843c0-7.028,2.218-13.083,6.651-18.167c4.434-5.083,10.112-7.623,17.032-7.623h72.021v19.464  h-72.021c-2.703,0-4.055,2.109-4.055,6.326c0,4.109,1.352,6.164,4.055,6.164h52.394c6.92,0,12.652,2.596,17.193,7.787  C562.22,332.85,564.329,338.852,564.329,345.88z"
+   id="path3277" />
+<polygon
+   fill="#BA160C"
+   points="661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 656.952,339.555 591.906,339.555   591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 "
+   id="polygon3279" />
+</svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbase_logo_with_orca.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbase_logo_with_orca.png b/src/site/resources/images/hbase_logo_with_orca.png
new file mode 100644
index 0000000..7ed60e2
Binary files /dev/null and b/src/site/resources/images/hbase_logo_with_orca.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbase_logo_with_orca.xcf
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbase_logo_with_orca.xcf b/src/site/resources/images/hbase_logo_with_orca.xcf
new file mode 100644
index 0000000..8d88da2
Binary files /dev/null and b/src/site/resources/images/hbase_logo_with_orca.xcf differ

[12/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
HBASE-15042 Moved site materials to standard Maven place

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 7da47509d2c04949b7a685823dfe36b86b9a4b53
Parents: 62deb81
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Mon Jul 3 14:33:01 2017 +0200
Committer: Michael Stack <st...@apache.org>
Committed: Sat Aug 5 15:22:46 2017 +0800

----------------------------------------------------------------------
 dev-support/hbase-personality.sh                |   2 +-
 pom.xml                                         |  14 +-
 .../appendix_contributing_to_documentation.adoc |   6 +-
 src/main/asciidoc/_chapters/images              |   2 +-
 src/main/asciidoc/images                        |   2 +-
 src/main/site/asciidoc/acid-semantics.adoc      | 118 ---
 src/main/site/asciidoc/bulk-loads.adoc          |  23 -
 src/main/site/asciidoc/cygwin.adoc              | 197 -----
 src/main/site/asciidoc/export_control.adoc      |  44 --
 src/main/site/asciidoc/index.adoc               |  75 --
 src/main/site/asciidoc/metrics.adoc             | 102 ---
 src/main/site/asciidoc/old_news.adoc            | 121 ----
 src/main/site/asciidoc/pseudo-distributed.adoc  |  23 -
 src/main/site/asciidoc/replication.adoc         |  22 -
 src/main/site/asciidoc/resources.adoc           |  27 -
 src/main/site/asciidoc/sponsors.adoc            |  36 -
 .../site/custom/project-info-report.properties  | 303 --------
 src/main/site/resources/.htaccess               |   8 -
 src/main/site/resources/book/.empty             |   1 -
 src/main/site/resources/css/site.css            | 118 ---
 src/main/site/resources/doap_Hbase.rdf          |  57 --
 src/main/site/resources/images/architecture.gif | Bin 15461 -> 0 bytes
 src/main/site/resources/images/bc_basic.png     | Bin 239294 -> 0 bytes
 src/main/site/resources/images/bc_config.png    | Bin 124066 -> 0 bytes
 src/main/site/resources/images/bc_l1.png        | Bin 91603 -> 0 bytes
 .../site/resources/images/bc_l2_buckets.png     | Bin 143801 -> 0 bytes
 src/main/site/resources/images/bc_stats.png     | Bin 111566 -> 0 bytes
 src/main/site/resources/images/big_h_logo.png   | Bin 2286 -> 0 bytes
 src/main/site/resources/images/big_h_logo.svg   | 139 ----
 .../images/data_block_diff_encoding.png         | Bin 54479 -> 0 bytes
 .../resources/images/data_block_no_encoding.png | Bin 46836 -> 0 bytes
 .../images/data_block_prefix_encoding.png       | Bin 35271 -> 0 bytes
 src/main/site/resources/images/favicon.ico      | Bin 1150 -> 0 bytes
 src/main/site/resources/images/hadoop-logo.jpg  | Bin 9443 -> 0 bytes
 src/main/site/resources/images/hbase_logo.png   | Bin 2997 -> 0 bytes
 src/main/site/resources/images/hbase_logo.svg   |  78 --
 .../resources/images/hbase_logo_with_orca.png   | Bin 11618 -> 0 bytes
 .../resources/images/hbase_logo_with_orca.xcf   | Bin 84265 -> 0 bytes
 .../images/hbase_logo_with_orca_large.png       | Bin 21196 -> 0 bytes
 .../images/hbase_replication_diagram.jpg        | Bin 52298 -> 0 bytes
 .../resources/images/hbasecon2015.30percent.png | Bin 8684 -> 0 bytes
 .../images/hbasecon2016-stack-logo.jpg          | Bin 32105 -> 0 bytes
 .../resources/images/hbasecon2016-stacked.png   | Bin 24924 -> 0 bytes
 src/main/site/resources/images/hbasecon2017.png | Bin 3982 -> 0 bytes
 .../site/resources/images/hbaseconasia2017.png  | Bin 23656 -> 0 bytes
 src/main/site/resources/images/hfile.png        | Bin 33661 -> 0 bytes
 src/main/site/resources/images/hfilev2.png      | Bin 57858 -> 0 bytes
 .../resources/images/jumping-orca_rotated.png   | Bin 52812 -> 0 bytes
 .../resources/images/jumping-orca_rotated.xcf   | Bin 77560 -> 0 bytes
 .../images/jumping-orca_rotated_12percent.png   | Bin 2401 -> 0 bytes
 .../images/jumping-orca_rotated_25percent.png   | Bin 4780 -> 0 bytes
 .../images/jumping-orca_transparent_rotated.xcf | Bin 135399 -> 0 bytes
 .../resources/images/region_split_process.png   | Bin 338255 -> 0 bytes
 .../site/resources/images/region_states.png     | Bin 99146 -> 0 bytes
 .../resources/images/replication_overview.png   | Bin 207537 -> 0 bytes
 .../resources/images/timeline_consistency.png   | Bin 88301 -> 0 bytes
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar   | Bin 344936 -> 0 bytes
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom   | 718 -------------------
 .../maven-fluido-skin/maven-metadata-local.xml  |  12 -
 src/main/site/site.xml                          | 131 ----
 src/main/site/xdoc/acid-semantics.xml           | 235 ------
 src/main/site/xdoc/bulk-loads.xml               |  34 -
 src/main/site/xdoc/coc.xml                      |  92 ---
 src/main/site/xdoc/cygwin.xml                   | 245 -------
 src/main/site/xdoc/export_control.xml           |  59 --
 src/main/site/xdoc/index.xml                    | 109 ---
 src/main/site/xdoc/metrics.xml                  | 150 ----
 src/main/site/xdoc/old_news.xml                 |  92 ---
 src/main/site/xdoc/poweredbyhbase.xml           | 398 ----------
 src/main/site/xdoc/pseudo-distributed.xml       |  42 --
 src/main/site/xdoc/replication.xml              |  35 -
 src/main/site/xdoc/resources.xml                |  45 --
 src/main/site/xdoc/sponsors.xml                 |  50 --
 src/main/site/xdoc/supportingprojects.xml       | 161 -----
 src/site/asciidoc/acid-semantics.adoc           | 118 +++
 src/site/asciidoc/bulk-loads.adoc               |  23 +
 src/site/asciidoc/cygwin.adoc                   | 197 +++++
 src/site/asciidoc/export_control.adoc           |  44 ++
 src/site/asciidoc/index.adoc                    |  75 ++
 src/site/asciidoc/metrics.adoc                  | 102 +++
 src/site/asciidoc/old_news.adoc                 | 121 ++++
 src/site/asciidoc/pseudo-distributed.adoc       |  23 +
 src/site/asciidoc/replication.adoc              |  22 +
 src/site/asciidoc/resources.adoc                |  27 +
 src/site/asciidoc/sponsors.adoc                 |  36 +
 src/site/custom/project-info-report.properties  | 303 ++++++++
 src/site/resources/.htaccess                    |   8 +
 src/site/resources/book/.empty                  |   1 +
 src/site/resources/css/site.css                 | 118 +++
 src/site/resources/doap_Hbase.rdf               |  57 ++
 src/site/resources/images/architecture.gif      | Bin 0 -> 15461 bytes
 src/site/resources/images/bc_basic.png          | Bin 0 -> 239294 bytes
 src/site/resources/images/bc_config.png         | Bin 0 -> 124066 bytes
 src/site/resources/images/bc_l1.png             | Bin 0 -> 91603 bytes
 src/site/resources/images/bc_l2_buckets.png     | Bin 0 -> 143801 bytes
 src/site/resources/images/bc_stats.png          | Bin 0 -> 111566 bytes
 src/site/resources/images/big_h_logo.png        | Bin 0 -> 2286 bytes
 src/site/resources/images/big_h_logo.svg        | 139 ++++
 .../images/data_block_diff_encoding.png         | Bin 0 -> 54479 bytes
 .../resources/images/data_block_no_encoding.png | Bin 0 -> 46836 bytes
 .../images/data_block_prefix_encoding.png       | Bin 0 -> 35271 bytes
 src/site/resources/images/favicon.ico           | Bin 0 -> 1150 bytes
 src/site/resources/images/hadoop-logo.jpg       | Bin 0 -> 9443 bytes
 src/site/resources/images/hbase_logo.png        | Bin 0 -> 2997 bytes
 src/site/resources/images/hbase_logo.svg        |  78 ++
 .../resources/images/hbase_logo_with_orca.png   | Bin 0 -> 11618 bytes
 .../resources/images/hbase_logo_with_orca.xcf   | Bin 0 -> 84265 bytes
 .../images/hbase_logo_with_orca_large.png       | Bin 0 -> 21196 bytes
 .../images/hbase_replication_diagram.jpg        | Bin 0 -> 52298 bytes
 .../resources/images/hbasecon2015.30percent.png | Bin 0 -> 8684 bytes
 .../images/hbasecon2016-stack-logo.jpg          | Bin 0 -> 32105 bytes
 .../resources/images/hbasecon2016-stacked.png   | Bin 0 -> 24924 bytes
 src/site/resources/images/hbasecon2017.png      | Bin 0 -> 3982 bytes
 src/site/resources/images/hbaseconasia2017.png  | Bin 0 -> 23656 bytes
 src/site/resources/images/hfile.png             | Bin 0 -> 33661 bytes
 src/site/resources/images/hfilev2.png           | Bin 0 -> 57858 bytes
 .../resources/images/jumping-orca_rotated.png   | Bin 0 -> 52812 bytes
 .../resources/images/jumping-orca_rotated.xcf   | Bin 0 -> 77560 bytes
 .../images/jumping-orca_rotated_12percent.png   | Bin 0 -> 2401 bytes
 .../images/jumping-orca_rotated_25percent.png   | Bin 0 -> 4780 bytes
 .../images/jumping-orca_transparent_rotated.xcf | Bin 0 -> 135399 bytes
 .../resources/images/region_split_process.png   | Bin 0 -> 338255 bytes
 src/site/resources/images/region_states.png     | Bin 0 -> 99146 bytes
 .../resources/images/replication_overview.png   | Bin 0 -> 207537 bytes
 .../resources/images/timeline_consistency.png   | Bin 0 -> 88301 bytes
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar   | Bin 0 -> 344936 bytes
 .../1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom   | 718 +++++++++++++++++++
 .../maven-fluido-skin/maven-metadata-local.xml  |  12 +
 src/site/site.xml                               | 131 ++++
 src/site/xdoc/acid-semantics.xml                | 235 ++++++
 src/site/xdoc/bulk-loads.xml                    |  34 +
 src/site/xdoc/coc.xml                           |  92 +++
 src/site/xdoc/cygwin.xml                        | 245 +++++++
 src/site/xdoc/export_control.xml                |  59 ++
 src/site/xdoc/index.xml                         | 109 +++
 src/site/xdoc/metrics.xml                       | 150 ++++
 src/site/xdoc/old_news.xml                      |  92 +++
 src/site/xdoc/poweredbyhbase.xml                | 398 ++++++++++
 src/site/xdoc/pseudo-distributed.xml            |  42 ++
 src/site/xdoc/replication.xml                   |  35 +
 src/site/xdoc/resources.xml                     |  45 ++
 src/site/xdoc/sponsors.xml                      |  50 ++
 src/site/xdoc/supportingprojects.xml            | 161 +++++
 143 files changed, 4113 insertions(+), 4113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 7a9ce3e..9f591b1 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -393,7 +393,7 @@ function mvnsite_filefilter
   local filename=$1
 
   if [[ ${BUILDTOOL} = maven ]]; then
-    if [[ ${filename} =~ src/main/site || ${filename} =~ src/main/asciidoc ]]; then
+    if [[ ${filename} =~ src/site || ${filename} =~ src/main/asciidoc ]]; then
       yetus_debug "tests/mvnsite: ${filename}"
       add_test mvnsite
     fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f3b3f36..4026d81 100644
--- a/pom.xml
+++ b/pom.xml
@@ -871,7 +871,7 @@
               <exclude>.svn/**</exclude>
               <exclude>**/.settings/**</exclude>
               <exclude>**/patchprocess/**</exclude>
-              <exclude>src/main/site/resources/repo/**</exclude>
+              <exclude>src/site/resources/repo/**</exclude>
               <exclude>**/dependency-reduced-pom.xml</exclude>
               <exclude>**/rat.txt</exclude>
               <!-- exclude the shaded protobuf files -->
@@ -1161,8 +1161,8 @@
           </dependency>
         </dependencies>
         <configuration>
-          <siteDirectory>${basedir}/src/main/site</siteDirectory>
-          <customBundle>${basedir}/src/main/site/custom/project-info-report.properties</customBundle>
+          <siteDirectory>${basedir}/src/site</siteDirectory>
+          <customBundle>${basedir}/src/site/custom/project-info-report.properties</customBundle>
           <inputEncoding>UTF-8</inputEncoding>
           <outputEncoding>UTF-8</outputEncoding>
         </configuration>
@@ -1237,7 +1237,7 @@
               <outputDirectory>${project.reporting.outputDirectory}/</outputDirectory>
               <resources>
                 <resource>
-                  <directory>${basedir}/src/main/site/resources/</directory>
+                  <directory>${basedir}/src/site/resources/</directory>
                   <includes>
                     <include>.htaccess</include>
                   </includes>
@@ -1256,7 +1256,7 @@
               <outputDirectory>${project.reporting.outputDirectory}/</outputDirectory>
               <resources>
                 <resource>
-                  <directory>${basedir}/src/main/site/resources/</directory>
+                  <directory>${basedir}/src/site/resources/</directory>
                   <includes>
                     <include>book/**</include>
                   </includes>
@@ -3170,7 +3170,7 @@
             </reports>
           </reportSet>
         </reportSets>
-        <!-- see src/main/site/site.xml for selected reports -->
+        <!-- see src/site/site.xml for selected reports -->
         <configuration>
           <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
         </configuration>
@@ -3416,7 +3416,7 @@
     <repository>
         <id>project.local</id>
         <name>project</name>
-        <url>file:${project.basedir}/src/main/site/resources/repo</url>
+        <url>file:${project.basedir}/src/site/resources/repo</url>
     </repository>
 </repositories>
 </project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc b/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
index 0337182..ec9282b 100644
--- a/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
+++ b/src/main/asciidoc/_chapters/appendix_contributing_to_documentation.adoc
@@ -119,7 +119,7 @@ JIRA and add a version number to the name of the new patch.
 
 === Editing the HBase Website
 
-The source for the HBase website is in the HBase source, in the _src/main/site/_ directory.
+The source for the HBase website is in the HBase source, in the _src/site/_ directory.
 Within this directory, source for the individual pages is in the _xdocs/_ directory,
 and images referenced in those pages are in the _resources/images/_ directory.
 This directory also stores images used in the HBase Reference Guide.
@@ -216,7 +216,7 @@ link:http://www.google.com[Google]
 ----
 image::sunset.jpg[Alt Text]
 ----
-(put the image in the src/main/site/resources/images directory)
+(put the image in the src/site/resources/images directory)
 | An inline image | The image with alt text, as part of the text flow |
 ----
 image:sunset.jpg [Alt Text]
@@ -389,7 +389,7 @@ Inline images cannot have titles. They are generally small images like GUI butto
 image:sunset.jpg[Alt Text]
 ----
 
-When doing a local build, save the image to the _src/main/site/resources/images/_ directory.
+When doing a local build, save the image to the _src/site/resources/images/_ directory.
 When you link to the image, do not include the directory portion of the path.
 The image will be copied to the appropriate target location during the build of the output.
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/asciidoc/_chapters/images
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/images b/src/main/asciidoc/_chapters/images
index 1e0c6c1..dc4cd20 120000
--- a/src/main/asciidoc/_chapters/images
+++ b/src/main/asciidoc/_chapters/images
@@ -1 +1 @@
-../../site/resources/images
\ No newline at end of file
+../../../site/resources/images/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/asciidoc/images
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/images b/src/main/asciidoc/images
index 06d04d0..02e8e94 120000
--- a/src/main/asciidoc/images
+++ b/src/main/asciidoc/images
@@ -1 +1 @@
-../site/resources/images
\ No newline at end of file
+../../site/resources/images/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/acid-semantics.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/acid-semantics.adoc b/src/main/site/asciidoc/acid-semantics.adoc
deleted file mode 100644
index 0038901..0000000
--- a/src/main/site/asciidoc/acid-semantics.adoc
+++ /dev/null
@@ -1,118 +0,0 @@
-////
-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.
-////
-
-= Apache HBase (TM) ACID Properties
-
-== About this Document
-
-Apache HBase (TM) is not an ACID compliant database. However, it does guarantee certain specific properties.
-
-This specification enumerates the ACID properties of HBase.
-
-== Definitions
-
-For the sake of common vocabulary, we define the following terms:
-Atomicity::
-  An operation is atomic if it either completes entirely or not at all.
-
-Consistency::
-  All actions cause the table to transition from one valid state directly to another (eg a row will not disappear during an update, etc).
-
-Isolation::
-  an operation is isolated if it appears to complete independently of any other concurrent transaction.
-
-Durability::
-  Any update that reports &quot;successful&quot; to the client will not be lost.
-
-Visibility::
-  An update is considered visible if any subsequent read will see the update as having been committed.
-
-
-The terms _must_ and _may_ are used as specified by link:[RFC 2119].
-
-In short, the word &quot;must&quot; implies that, if some case exists where the statement is not true, it is a bug. The word _may_ implies that, even if the guarantee is provided in a current release, users should not rely on it.
-
-== APIs to Consider
-- Read APIs
-* get
-* scan
-- Write APIs
-* put
-* batch put
-* delete
-- Combination (read-modify-write) APIs
-* incrementColumnValue
-* checkAndPut
-
-== Guarantees Provided
-
-.Atomicity
-.  All mutations are atomic within a row. Any put will either wholely succeed or wholely fail.footnoteref[Puts will either wholely succeed or wholely fail, provided that they are actually sent to the RegionServer.  If the writebuffer is used, Puts will not be sent until the writebuffer is filled or it is explicitly flushed.]
-.. An operation that returns a _success_ code has completely succeeded.
-.. An operation that returns a _failure_ code has completely failed.
-.. An operation that times out may have succeeded and may have failed. However, it will not have partially succeeded or failed.
-. This is true even if the mutation crosses multiple column families within a row.
-. APIs that mutate several rows will _not_ be atomic across the multiple rows. For example, a multiput that operates on rows 'a','b', and 'c' may return having mutated some but not all of the rows. In such cases, these APIs will return a list of success codes, each of which may be succeeded, failed, or timed out as described above.
-. The checkAndPut API happens atomically like the typical _compareAndSet (CAS)_ operation found in many hardware architectures.
-. The order of mutations is seen to happen in a well-defined order for each row, with no interleaving. For example, if one writer issues the mutation `a=1,b=1,c=1` and another writer issues the mutation `a=2,b=2,c=`, the row must either be `a=1,b=1,c=1` or `a=2,b=2,c=2` and must *not* be something like `a=1,b=2,c=1`. +
-NOTE:This is not true _across rows_ for multirow batch mutations.
-
-== Consistency and Isolation
-. All rows returned via any access API will consist of a complete row that existed at some point in the table's history.
-. This is true across column families - i.e a get of a full row that occurs concurrent with some mutations 1,2,3,4,5 will return a complete row that existed at some point in time between mutation i and i+1 for some i between 1 and 5.
-. The state of a row will only move forward through the history of edits to it.
-
-== Consistency of Scans
-A scan is *not* a consistent view of a table. Scans do *not* exhibit _snapshot isolation_.
-
-Rather, scans have the following properties:
-. Any row returned by the scan will be a consistent view (i.e. that version of the complete row existed at some point in time)footnoteref[consistency,A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of a row in one RPC then going back to fetch another portion of the row in a subsequent RPC. Intra-row scanning happens when you set a limit on how many values to return per Scan#next (See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)"[Scan#setBatch(int)]).]
-. A scan will always reflect a view of the data _at least as new as_ the beginning of the scan. This satisfies the visibility guarantees enumerated below.
-.. For example, if client A writes data X and then communicates via a side channel to client B, any scans started by client B will contain data at least as new as X.
-.. A scan _must_ reflect all mutations committed prior to the construction of the scanner, and _may_ reflect some mutations committed subsequent to the construction of the scanner.
-.. Scans must include _all_ data written prior to the scan (except in the case where data is subsequently mutated, in which case it _may_ reflect the mutation)
-
-Those familiar with relational databases will recognize this isolation level as "read committed".
-
-NOTE: The guarantees listed above regarding scanner consistency are referring to "transaction commit time", not the "timestamp" field of each cell. That is to say, a scanner started at time _t_ may see edits with a timestamp value greater than _t_, if those edits were committed with a "forward dated" timestamp before the scanner was constructed.
-
-== Visibility
-
-. When a client receives a &quot;success&quot; response for any mutation, that mutation is immediately visible to both that client and any client with whom it later communicates through side channels.footnoteref[consistency]
-. A row must never exhibit so-called "time-travel" properties. That is to say, if a series of mutations moves a row sequentially through a series of states, any sequence of concurrent reads will return a subsequence of those states. +
-For example, if a row's cells are mutated using the `incrementColumnValue` API, a client must never see the value of any cell decrease. +
-This is true regardless of which read API is used to read back the mutation.
-. Any version of a cell that has been returned to a read operation is guaranteed to be durably stored.
-
-== Durability
-. All visible data is also durable data. That is to say, a read will never return data that has not been made durable on disk.footnoteref[durability,In the context of Apache HBase, _durably on disk_; implies an `hflush()` call on the transaction log. This does not actually imply an `fsync()` to magnetic media, but rather just that the data has been written to the OS cache on all replicas of the log. In the case of a full datacenter power loss, it is possible that the edits are not truly durable.]
-. Any operation that returns a &quot;success&quot; code (eg does not throw an exception) will be made durable.footnoteref[durability]
-. Any operation that returns a &quot;failure&quot; code will not be made durable (subject to the Atomicity guarantees above).
-. All reasonable failure scenarios will not affect any of the guarantees of this document.
-
-== Tunability
-
-All of the above guarantees must be possible within Apache HBase. For users who would like to trade off some guarantees for performance, HBase may offer several tuning options. For example:
-
-* Visibility may be tuned on a per-read basis to allow stale reads or time travel.
-* Durability may be tuned to only flush data to disk on a periodic basis.
-
-== More Information
-
-For more information, see the link:book.html#client[client architecture] and  link:book.html#datamodel[data model] sections in the Apache HBase Reference Guide. 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/bulk-loads.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/bulk-loads.adoc b/src/main/site/asciidoc/bulk-loads.adoc
deleted file mode 100644
index fc320d8..0000000
--- a/src/main/site/asciidoc/bulk-loads.adoc
+++ /dev/null
@@ -1,23 +0,0 @@
-////
-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.
-////
-
-= Bulk Loads in Apache HBase (TM)
-
-This page has been retired.  The contents have been moved to the link:book.html#arch.bulk.load[Bulk Loading] section in the Reference Guide.
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/cygwin.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/cygwin.adoc b/src/main/site/asciidoc/cygwin.adoc
deleted file mode 100644
index 11c4df4..0000000
--- a/src/main/site/asciidoc/cygwin.adoc
+++ /dev/null
@@ -1,197 +0,0 @@
-////
-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.
-////
-
-
-== Installing Apache HBase (TM) on Windows using Cygwin
-
-== Introduction
-
-link:http://hbase.apache.org[Apache HBase (TM)] is a distributed, column-oriented store, modeled after Google's link:http://research.google.com/archive/bigtable.html[BigTable]. Apache HBase is built on top of link:http://hadoop.apache.org[Hadoop] for its link:http://hadoop.apache.org/mapreduce[MapReduce] link:http://hadoop.apache.org/hdfs[distributed file system] implementations. All these projects are open-source and part of the link:http://www.apache.org[Apache Software Foundation].
-
-== Purpose
-
-This document explains the *intricacies* of running Apache HBase on Windows using Cygwin* as an all-in-one single-node installation for testing and development. The HBase link:http://hbase.apache.org/apidocs/overview-summary.html#overview_description[Overview] and link:book.html#getting_started[QuickStart] guides on the other hand go a long way in explaning how to setup link:http://hadoop.apache.org/hbase[HBase] in more complex deployment scenarios.
-
-== Installation
-
-For running Apache HBase on Windows, 3 technologies are required: 
-* Java
-* Cygwin
-* SSH 
-
-The following paragraphs detail the installation of each of the aforementioned technologies.
-
-=== Java
-
-HBase depends on the link:http://java.sun.com/javase/6/[Java Platform, Standard Edition, 6 Release]. So the target system has to be provided with at least the Java Runtime Environment (JRE); however if the system will also be used for development, the Jave Development Kit (JDK) is preferred. You can download the latest versions for both from link:http://java.sun.com/javase/downloads/index.jsp[Sun's download page]. Installation is a simple GUI wizard that guides you through the process.
-
-=== Cygwin
-
-Cygwin is probably the oddest technology in this solution stack. It provides a dynamic link library that emulates most of a *nix environment on Windows. On top of that a whole bunch of the most common *nix tools are supplied. Combined, the DLL with the tools form a very *nix-alike environment on Windows.
-
-For installation, Cygwin provides the link:http://cygwin.com/setup.exe[`setup.exe` utility] that tracks the versions of all installed components on the target system and provides the mechanism for installing or updating everything from the mirror sites of Cygwin.
-
-To support installation, the `setup.exe` utility uses 2 directories on the target system. The *Root* directory for Cygwin (defaults to _C:\cygwin)_ which will become _/_ within the eventual Cygwin installation; and the *Local Package* directory (e.g. _C:\cygsetup_ that is the cache where `setup.exe`stores the packages before they are installed. The cache must not be the same folder as the Cygwin root.
-
-Perform following steps to install Cygwin, which are elaboratly detailed in the link:http://cygwin.com/cygwin-ug-net/setup-net.html[2nd chapter] of the link:http://cygwin.com/cygwin-ug-net/cygwin-ug-net.html[Cygwin User's Guide].
-
-. Make sure you have `Administrator` privileges on the target system.
-. Choose and create you Root and *Local Package* directories. A good suggestion is to use `C:\cygwin\root` and `C:\cygwin\setup` folders.
-. Download the `setup.exe` utility and save it to the *Local Package* directory. Run the `setup.exe` utility.
-.. Choose  the `Install from Internet` option.
-.. Choose your *Root* and *Local Package* folders.
-.. Select an appropriate mirror.
-.. Don't select any additional packages yet, as we only want to install Cygwin for now.
-.. Wait for download and install.
-.. Finish the installation.
-. Optionally, you can now also add a shortcut to your Start menu pointing to the `setup.exe` utility in the *Local Package *folder.
-. Add `CYGWIN_HOME` system-wide environment variable that points to your *Root* directory.
-. Add `%CYGWIN_HOME%\bin` to the end of your `PATH` environment variable.
-. Reboot the sytem after making changes to the environment variables otherwise the OS will not be able to find the Cygwin utilities.
-. Test your installation by running your freshly created shortcuts or the `Cygwin.bat` command in the *Root* folder. You should end up in a terminal window that is running a link:http://www.gnu.org/software/bash/manual/bashref.html[Bash shell]. Test the shell by issuing following commands:
-.. `cd /` should take you to thr *Root* directory in Cygwin.
-.. The `LS` commands that should list all files and folders in the current directory.
-.. Use the `exit` command to end the terminal.
-. When needed, to *uninstall* Cygwin you can simply delete the *Root* and *Local Package* directory, and the *shortcuts* that were created during installation.
-
-=== SSH
-
-HBase (and Hadoop) rely on link:http://nl.wikipedia.org/wiki/Secure_Shell[*SSH*] for interprocess/-node *communication* and launching* remote commands*. SSH will be provisioned on the target system via Cygwin, which supports running Cygwin programs as *Windows services*!
-
-. Rerun the `*setup.exe*`* utility*.
-. Leave all parameters as is, skipping through the wizard using the `Next` button until the `Select Packages` panel is shown.
-. Maximize the window and click the `View` button to toggle to the list view, which is ordered alfabetically on `Package`, making it easier to find the packages we'll need.
-. Select the following packages by clicking the status word (normally `Skip`) so it's marked for installation. Use the `Next `button to download and install the packages.
-.. `OpenSSH`
-.. `tcp_wrappers`
-.. `diffutils`
-.. `zlib`
-. Wait for the install to complete and finish the installation.
-
-=== HBase
-
-Download the *latest release* of Apache HBase from link:http://www.apache.org/dyn/closer.cgi/hbase/. As the Apache HBase distributable is just a zipped archive, installation is as simple as unpacking the archive so it ends up in its final *installation* directory. Notice that HBase has to be installed in Cygwin and a good directory suggestion is to use `/usr/local/` (or [`*Root* directory]\usr\local` in Windows slang). You should end up with a `/usr/local/hbase-_versi` installation in Cygwin.
-
-This finishes installation. We go on with the configuration.
-
-== Configuration
-
-There are 3 parts left to configure: *Java, SSH and HBase* itself. Following paragraphs explain eacht topic in detail.
-
-=== Java
-
-One important thing to remember in shell scripting in general (i.e. *nix and Windows) is that managing, manipulating and assembling path names that contains spaces can be very hard, due to the need to escape and quote those characters and strings. So we try to stay away from spaces in path names. *nix environments can help us out here very easily by using *symbolic links*.
-
-. Create a link in `/usr/local` to the Java home directory by using the following command and substituting the name of your chosen Java environment: +
-----
-LN -s /cygdrive/c/Program\ Files/Java/*_jre name_*/usr/local/*_jre name_*
-----
-. Test your java installation by changing directories to your Java folder `CD /usr/local/_jre name_` and issueing the command `./bin/java -version`. This should output your version of the chosen JRE.
-
-=== SSH 
-
-Configuring *SSH *is quite elaborate, but primarily a question of launching it by default as a* Windows service*.
-
-. On Windows Vista and above make sure you run the Cygwin shell with *elevated privileges*, by right-clicking on the shortcut an using `Run as Administrator`.
-. First of all, we have to make sure the *rights on some crucial files* are correct. Use the commands underneath. You can verify all rights by using the `LS -L` command on the different files. Also, notice the auto-completion feature in the shell using `TAB` is extremely handy in these situations.
-.. `chmod +r /etc/passwd` to make the passwords file readable for all
-.. `chmod u+w /etc/passwd` to make the passwords file writable for the owner
-.. `chmod +r /etc/group` to make the groups file readable for all
-.. `chmod u+w /etc/group` to make the groups file writable for the owner
-.. `chmod 755 /var` to make the var folder writable to owner and readable and executable to all
-. Edit the */etc/hosts.allow* file using your favorite editor (why not VI in the shell!) and make sure the following two lines are in there before the `PARANOID` line: +
-----
-ALL : localhost 127.0.0.1/32 : allow
-ALL : [::1]/128 : allow
-----
-. Next we have to *configure SSH* by using the script `ssh-host-config`.
-.. If this script asks to overwrite an existing `/etc/ssh_config`, answer `yes`.
-.. If this script asks to overwrite an existing `/etc/sshd_config`, answer `yes`.
-.. If this script asks to use privilege separation, answer `yes`.
-.. If this script asks to install `sshd` as a service, answer `yes`. Make sure you started your shell as Adminstrator!
-.. If this script asks for the CYGWIN value, just `enter` as the default is `ntsec`.
-.. If this script asks to create the `sshd` account, answer `yes`.
-.. If this script asks to use a different user name as service account, answer `no` as the default will suffice.
-.. If this script asks to create the `cyg_server` account, answer `yes`. Enter a password for the account.
-. *Start the SSH service* using `net start sshd` or `cygrunsrv  --start  sshd`. Notice that `cygrunsrv` is the utility that make the process run as a Windows service. Confirm that you see a message stating that `the CYGWIN sshd service  was started succesfully.`
-. Harmonize Windows and Cygwin* user account* by using the commands: +
-----
-mkpasswd -cl > /etc/passwd
-mkgroup --local > /etc/group
-----
-. Test *the installation of SSH:
-.. Open a new Cygwin terminal.
-.. Use the command `whoami` to verify your userID.
-.. Issue an `ssh localhost` to connect to the system itself.
-.. Answer `yes` when presented with the server's fingerprint.
-.. Issue your password when prompted.
-.. Test a few commands in the remote session
-.. The `exit` command should take you back to your first shell in Cygwin.
-. `Exit` should terminate the Cygwin shell.
-
-=== HBase
-
-If all previous configurations are working properly, we just need some tinkering at the *HBase config* files to properly resolve on Windows/Cygwin. All files and paths referenced here start from the HBase `[*installation* directory]` as working directory.
-
-. HBase uses the `./conf/*hbase-env.sh*` to configure its dependencies on the runtime environment. Copy and uncomment following lines just underneath their original, change them to fit your environemnt. They should read something like: +
-----
-export JAVA_HOME=/usr/local/_jre name_
-export HBASE_IDENT_STRING=$HOSTNAME
-----
-. HBase uses the _./conf/`*hbase-default.xml*`_ file for configuration. Some properties do not resolve to existing directories because the JVM runs on Windows. This is the major issue to keep in mind when working with Cygwin: within the shell all paths are *nix-alike, hence relative to the root `/`. However, every parameter that is to be consumed within the windows processes themself, need to be Windows settings, hence `C:\`-alike. Change following propeties in the configuration file, adjusting paths where necessary to conform with your own installation:
-.. `hbase.rootdir` must read e.g. `file:///C:/cygwin/root/tmp/hbase/data`
-.. `hbase.tmp.dir` must read `C:/cygwin/root/tmp/hbase/tmp`
-.. `hbase.zookeeper.quorum` must read `127.0.0.1` because for some reason `localhost` doesn't seem to resolve properly on Cygwin.
-. Make sure the configured `hbase.rootdir` and `hbase.tmp.dir` *directories exist* and have the proper* rights* set up e.g. by issuing a `chmod 777` on them.
-
-== Testing
-
-This should conclude the installation and configuration of Apache HBase on Windows using Cygwin. So it's time *to test it*.
-
-. Start a Cygwin* terminal*, if you haven't already.
-. Change directory to HBase *installation* using `CD /usr/local/hbase-_version_`, preferably using auto-completion.
-. *Start HBase* using the command `./bin/start-hbase.sh`
-.. When prompted to accept the SSH fingerprint, answer `yes`.
-.. When prompted, provide your password. Maybe multiple times.
-.. When the command completes, the HBase server should have started.
-.. However, to be absolutely certain, check the logs in the `./logs` directory for any exceptions.
-. Next we *start the HBase shell* using the command `./bin/hbase shell`
-. We run some simple *test commands*
-.. Create a simple table using command `create 'test', 'data'`
-.. Verify the table exists using the command `list`
-.. Insert data into the table using e.g. +
-----
-put 'test', 'row1', 'data:1', 'value1'
-put 'test', 'row2', 'data:2', 'value2'
-put 'test', 'row3', 'data:3', 'value3'
-----
-.. List all rows in the table using the command `scan 'test'` that should list all the rows previously inserted. Notice how 3 new columns where added without changing the schema!
-.. Finally we get rid of the table by issuing `disable 'test'` followed by `drop 'test'` and verified by `list` which should give an empty listing.
-. *Leave the shell* by `exit`
-. To *stop the HBase server* issue the `./bin/stop-hbase.sh` command. And wait for it to complete!!! Killing the process might corrupt your data on disk.
-. In case of *problems*,
-.. Verify the HBase logs in the `./logs` directory.
-.. Try to fix the problem
-.. Get help on the forums or IRC (`#hbase@freenode.net`). People are very active and keen to help out!
-.. Stop and retest the server.
-
-== Conclusion
-
-Now your *HBase *server is running, *start coding* and build that next killer app on this particular, but scalable datastore!
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/export_control.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/export_control.adoc b/src/main/site/asciidoc/export_control.adoc
deleted file mode 100644
index 1bbefb5..0000000
--- a/src/main/site/asciidoc/export_control.adoc
+++ /dev/null
@@ -1,44 +0,0 @@
-////
-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.
-////
-
-
-= Export Control
-
-This distribution uses or includes cryptographic software. The country in
-which you currently reside may have restrictions on the import, possession,
-use, and/or re-export to another country, of encryption software. BEFORE
-using any encryption software, please check your country's laws, regulations
-and policies concerning the import, possession, or use, and re-export of
-encryption software, to see if this is permitted. See the
-link:http://www.wassenaar.org/[Wassenaar Arrangement] for more
-information.
-
-The U.S. Government Department of Commerce, Bureau of Industry and Security 
-(BIS), has classified this software as Export Commodity Control Number (ECCN) 
-5D002.C.1, which includes information security software using or performing 
-cryptographic functions with asymmetric algorithms. The form and manner of this
-Apache Software Foundation distribution makes it eligible for export under the 
-License Exception ENC Technology Software Unrestricted (TSU) exception (see the
-BIS Export Administration Regulations, Section 740.13) for both object code and
-source code.
-
-Apache HBase uses the built-in java cryptography libraries. See Oracle's
-information regarding
-link:http://www.oracle.com/us/products/export/export-regulations-345813.html[Java cryptographic export regulations]
-for more details.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/index.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/index.adoc b/src/main/site/asciidoc/index.adoc
deleted file mode 100644
index 9b31c49..0000000
--- a/src/main/site/asciidoc/index.adoc
+++ /dev/null
@@ -1,75 +0,0 @@
-////
-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.
-////
-
-= Apache HBase&#153; Home
-
-.Welcome to Apache HBase(TM)
-link:http://www.apache.org/[Apache HBase(TM)] is the link:http://hadoop.apache.org[Hadoop] database, a distributed, scalable, big data store.
-
-.When Would I Use Apache HBase?
-Use Apache HBase when you need random, realtime read/write access to your Big Data. +
-This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters of commodity hardware.
-
-Apache HBase is an open-source, distributed, versioned, non-relational database modeled after Google's link:http://research.google.com/archive/bigtable.html[Bigtable: A Distributed Storage System for Structured Data] by Chang et al.
-
-Just as Bigtable leverages the distributed data storage provided by the Google File System, Apache HBase provides Bigtable-like capabilities on top of Hadoop and HDFS.
-
-.Features
-- Linear and modular scalability.
-- Strictly consistent reads and writes.
-- Automatic and configurable sharding of tables
-- Automatic failover support between RegionServers.
-- Convenient base classes for backing Hadoop MapReduce jobs with Apache HBase tables.
-- Easy to use Java API for client access.
-- Block cache and Bloom Filters for real-time queries.
-- Query predicate push down via server side Filters
-- Thrift gateway and a REST-ful Web service that supports XML, Protobuf, and binary data encoding options
-- Extensible jruby-based (JIRB) shell
-- Support for exporting metrics via the Hadoop metrics subsystem to files or Ganglia; or via JMX
-
-.Where Can I Get More Information?
-See the link:book.html#arch.overview[Architecture Overview], the link:book.html#faq[FAQ] and the other documentation links at the top!
-
-.Export Control
-The HBase distribution includes cryptographic software. See the link:export_control.html[export control notice].
-
-== News
-Feb 17, 2015:: link:http://www.meetup.com/hbaseusergroup/events/219260093/[HBase meetup around Strata+Hadoop World] in San Jose
-
-January 15th, 2015:: link:http://www.meetup.com/hbaseusergroup/events/218744798/[HBase meetup @ AppDynamics] in San Francisco
-
-November 20th, 2014::  link:http://www.meetup.com/hbaseusergroup/events/205219992/[HBase meetup @ WANdisco] in San Ramon
-
-October 27th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/207386102/[HBase Meetup @ Apple] in Cupertino
-
-October 15th, 2014:: link:http://www.meetup.com/HBase-NYC/events/207655552[HBase Meetup @ Google] on the night before Strata/HW in NYC
-
-September 25th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/203173692/[HBase Meetup @ Continuuity] in Palo Alto
-
-August 28th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/197773762/[HBase Meetup @ Sift Science] in San Francisco
-
-July 17th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/190994082/[HBase Meetup @ HP] in Sunnyvale
-
-June 5th, 2014:: link:http://www.meetup.com/Hadoop-Summit-Community-San-Jose/events/179081342/[HBase BOF at Hadoop Summit], San Jose Convention Center
-
-May 5th, 2014:: link:http://www.hbasecon.com[HBaseCon2014] at the Hilton San Francisco on Union Square
-
-March 12th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/160757912/[HBase Meetup @ Ancestry.com] in San Francisco
-
-View link:old_news.html[Old News]

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/metrics.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/metrics.adoc b/src/main/site/asciidoc/metrics.adoc
deleted file mode 100644
index be7d9a5..0000000
--- a/src/main/site/asciidoc/metrics.adoc
+++ /dev/null
@@ -1,102 +0,0 @@
-////
-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.
-////
-
-= Apache HBase (TM) Metrics
-
-== Introduction
-Apache HBase (TM) emits Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
-
-== Setup
-
-First read up on Hadoop link:http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html[metrics].
-
-If you are using ganglia, the link:http://wiki.apache.org/hadoop/GangliaMetrics[GangliaMetrics] wiki page is useful read.
-
-To have HBase emit metrics, edit `$HBASE_HOME/conf/hadoop-metrics.properties` and enable metric 'contexts' per plugin.  As of this writing, hadoop supports *file* and *ganglia* plugins. Yes, the hbase metrics files is named hadoop-metrics rather than _hbase-metrics_ because currently at least the hadoop metrics system has the properties filename hardcoded. Per metrics _context_, comment out the NullContext and enable one or more plugins instead.
-
-If you enable the _hbase_ context, on regionservers you'll see total requests since last
-metric emission, count of regions and storefiles as well as a count of memstore size.
-On the master, you'll see a count of the cluster's requests.
-
-Enabling the _rpc_ context is good if you are interested in seeing
-metrics on each hbase rpc method invocation (counts and time taken).
-
-The _jvm_ context is useful for long-term stats on running hbase jvms -- memory used, thread counts, etc. As of this writing, if more than one jvm is running emitting metrics, at least in ganglia, the stats are aggregated rather than reported per instance.
-
-== Using with JMX
-
-In addition to the standard output contexts supported by the Hadoop 
-metrics package, you can also export HBase metrics via Java Management 
-Extensions (JMX).  This will allow viewing HBase stats in JConsole or 
-any other JMX client.
-
-=== Enable HBase stats collection
-
-To enable JMX support in HBase, first edit `$HBASE_HOME/conf/hadoop-metrics.properties` to support metrics refreshing. (If you've running 0.94.1 and above, or have already configured `hadoop-metrics.properties` for another output context, you can skip this step).
-[source,bash]
-----
-# Configuration of the "hbase" context for null
-hbase.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-hbase.period=60
-
-# Configuration of the "jvm" context for null
-jvm.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-jvm.period=60
-
-# Configuration of the "rpc" context for null
-rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-rpc.period=60
-----
-
-=== Setup JMX Remote Access
-
-For remote access, you will need to configure JMX remote passwords and access profiles.  Create the files:
-`$HBASE_HOME/conf/jmxremote.passwd` (set permissions 
-        to 600):: +
-----
-monitorRole monitorpass
-controlRole controlpass
-----
-
-`$HBASE_HOME/conf/jmxremote.access`:: +
-----
-monitorRole readonly
-controlRole readwrite
-----
-
-=== Configure JMX in HBase startup
-
-Finally, edit the `$HBASE_HOME/conf/hbase-env.sh` script to add JMX support:
-[source,bash]
-----
-HBASE_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false"
-HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.password.file=$HBASE_HOME/conf/jmxremote.passwd"
-HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.access.file=$HBASE_HOME/conf/jmxremote.access"
-
-export HBASE_MASTER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10101"
-export HBASE_REGIONSERVER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10102"
-----
-
-After restarting the processes you want to monitor, you should now be able to run JConsole (included with the JDK since JDK 5.0) to view the statistics via JMX.  HBase MBeans are exported under the *`hadoop`* domain in JMX.
-
-
-== Understanding HBase Metrics
-
-For more information on understanding HBase metrics, see the link:book.html#hbase_metrics[metrics section] in the Apache HBase Reference Guide. 
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/old_news.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/old_news.adoc b/src/main/site/asciidoc/old_news.adoc
deleted file mode 100644
index ae44caa..0000000
--- a/src/main/site/asciidoc/old_news.adoc
+++ /dev/null
@@ -1,121 +0,0 @@
-////
-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.
-////
-
-= Old Apache HBase (TM) News
-
-February 10th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/163139322/[HBase Meetup @ Continuuity] in Palo Alto
-
-January 30th, 2014:: link:http://www.meetup.com/hbaseusergroup/events/158491762/[HBase Meetup @ Apple] in Cupertino
-
-January 30th, 2014:: link:http://www.meetup.com/Los-Angeles-HBase-User-group/events/160560282/[Los Angeles HBase User Group] in El Segundo
-
-October 24th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/140759692/[HBase User] and link:http://www.meetup.com/hackathon/events/144366512/[Developer] Meetup at HortonWorksin Palo Alto
-
-September 26, 2013:: link:http://www.meetup.com/hbaseusergroup/events/135862292/[HBase Meetup at Arista Networks] in San Francisco
-
-August 20th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/120534362/[HBase Meetup at Flurry] in San Francisco
-
-July 16th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/119929152/[HBase Meetup at Twitter] in San Francisco
-
-June 25th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/119154442/[Hadoop Summit Meetup].at San Jose Convention Center
-
-June 14th, 2013:: link:http://kijicon.eventbrite.com/[KijiCon: Building Big Data Apps] in San Francisco.
-
-June 13th, 2013:: link:http://www.hbasecon.com/[HBaseCon2013] in San Francisco.  Submit an Abstract!
-
-June 12th, 2013:: link:http://www.meetup.com/hackathon/events/123403802/[HBaseConHackAthon] at the Cloudera office in San Francisco.
-
-April 11th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/103587852/[HBase Meetup at AdRoll] in San Francisco
-
-February 28th, 2013:: link:http://www.meetup.com/hbaseusergroup/events/96584102/[HBase Meetup at Intel Mission Campus]
-
-February 19th, 2013:: link:http://www.meetup.com/hackathon/events/103633042/[Developers PowWow] at HortonWorks' new digs
-
-January 23rd, 2013:: link:http://www.meetup.com/hbaseusergroup/events/91381312/[HBase Meetup at WibiData World HQ!]
-
-December 4th, 2012:: link:http://www.meetup.com/hackathon/events/90536432/[0.96 Bug Squashing and Testing Hackathon] at Cloudera, SF.
-
-October 29th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/82791572/[HBase User Group Meetup] at Wize Commerce in San Mateo.
-
-October 25th, 2012:: link:http://www.meetup.com/HBase-NYC/events/81728932/[Strata/Hadoop World HBase Meetup.] in NYC
-
-September 11th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/80621872/[Contributor's Pow-Wow at HortonWorks HQ.]
-
-August 8th, 2012:: link:http://www.apache.org/dyn/closer.cgi/hbase/[Apache HBase 0.94.1 is available for download]
-
-June 15th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/59829652/[Birds-of-a-feather] in San Jose, day after:: link:http://hadoopsummit.org[Hadoop Summit]
-
-May 23rd, 2012:: link:http://www.meetup.com/hackathon/events/58953522/[HackConAthon] in Palo Alto
-
-May 22nd, 2012:: link:http://www.hbasecon.com[HBaseCon2012] in San Francisco
-
-March 27th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/56021562/[Meetup @ StumbleUpon] in San Francisco
-
-January 19th, 2012:: link:http://www.meetup.com/hbaseusergroup/events/46702842/[Meetup @ EBay]
-
-January 23rd, 2012:: Apache HBase 0.92.0 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
-
-December 23rd, 2011:: Apache HBase 0.90.5 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
-
-November 29th, 2011:: link:http://www.meetup.com/hackathon/events/41025972/[Developer Pow-Wow in SF] at Salesforce HQ
-
-November 7th, 2011:: link:http://www.meetup.com/hbaseusergroup/events/35682812/[HBase Meetup in NYC (6PM)] at the AppNexus office
-
-August 22nd, 2011:: link:http://www.meetup.com/hbaseusergroup/events/28518471/[HBase Hackathon (11AM) and Meetup (6PM)] at FB in PA
-
-June 30th, 2011:: link:http://www.meetup.com/hbaseusergroup/events/20572251/[HBase Contributor Day], the day after the:: link:http://developer.yahoo.com/events/hadoopsummit2011/[Hadoop Summit] hosted by Y!
-
-June 8th, 2011:: link:http://berlinbuzzwords.de/wiki/hbase-workshop-and-hackathon[HBase Hackathon] in Berlin to coincide with:: link:http://berlinbuzzwords.de/[Berlin Buzzwords]
-
-May 19th, 2011: Apache HBase 0.90.3 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
-
-April 12th, 2011: Apache HBase 0.90.2 released. link:http://www.apache.org/dyn/closer.cgi/hbase/[Download it!]
-
-March 21st, 2011:: link:http://www.meetup.com/hackathon/events/16770852/[HBase 0.92 Hackathon at StumbleUpon, SF]
-February 22nd, 2011:: link:http://www.meetup.com/hbaseusergroup/events/16492913/[HUG12: February HBase User Group at StumbleUpon SF]
-December 13th, 2010:: link:http://www.meetup.com/hackathon/calendar/15597555/[HBase Hackathon: Coprocessor Edition]
-November 19th, 2010:: link:http://huguk.org/[Hadoop HUG in London] is all about Apache HBase
-November 15-19th, 2010:: link:http://www.devoxx.com/display/Devoxx2K10/Home[Devoxx] features HBase Training and multiple HBase presentations
-
-October 12th, 2010:: HBase-related presentations by core contributors and users at:: link:http://www.cloudera.com/company/press-center/hadoop-world-nyc/[Hadoop World 2010]
-
-October 11th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/14606174/[HUG-NYC: HBase User Group NYC Edition] (Night before Hadoop World)
-June 30th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/13562846/[Apache HBase Contributor Workshop] (Day after Hadoop Summit)
-May 10th, 2010:: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project 
-
-April 19, 2010:: Signup for link:http://www.meetup.com/hbaseusergroup/calendar/12689490/[HBase User Group Meeting, HUG10] hosted by Trend Micro
-
-March 10th, 2010:: link:http://www.meetup.com/hbaseusergroup/calendar/12689351/[HBase User Group Meeting, HUG9] hosted by Mozilla
-
-January 27th, 2010:: Sign up for the link:http://www.meetup.com/hbaseusergroup/calendar/12241393/[HBase User Group Meeting, HUG8], at StumbleUpon in SF
-
-September 8th, 2010:: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release.  Get it off the link:http://www.apache.org/dyn/closer.cgi/hbase/[Releases] page.
-
-November 2-6th, 2009:: link:http://dev.us.apachecon.com/c/acus2009/[ApacheCon] in Oakland. The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.
-
-October 2nd, 2009:: HBase at Hadoop World in NYC. A few of us will be talking on Practical HBase out east at link:http://www.cloudera.com/hadoop-world-nyc[Hadoop World: NYC].
-
-August 7th-9th, 2009:: HUG7 and HBase Hackathon at StumbleUpon in SF: Sign up for the:: link:http://www.meetup.com/hbaseusergroup/calendar/10950511/[HBase User Group Meeting, HUG7] or for the link:http://www.meetup.com/hackathon/calendar/10951718/[Hackathon] or for both (all are welcome!).
-
-June, 2009::  HBase at HadoopSummit2009 and at NOSQL: See the link:http://wiki.apache.org/hadoop/HBase/HBasePresentations[presentations]
-
-March 3rd, 2009 :: HUG6 -- link:http://www.meetup.com/hbaseusergroup/calendar/9764004/[HBase User Group 6]
-
-January 30th, 2009:: LA Hbackathon: link:http://www.meetup.com/hbasela/calendar/9450876/[HBase January Hackathon Los Angeles] at link:http://streamy.com[Streamy] in Manhattan Beach
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/pseudo-distributed.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/pseudo-distributed.adoc b/src/main/site/asciidoc/pseudo-distributed.adoc
deleted file mode 100644
index d13c63b..0000000
--- a/src/main/site/asciidoc/pseudo-distributed.adoc
+++ /dev/null
@@ -1,23 +0,0 @@
-////
-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.
-////
-
-
-= Running Apache HBase (TM) in pseudo-distributed mode
-This page has been retired.  The contents have been moved to the link:book.html#distributed[Distributed Operation: Pseudo- and Fully-distributed modes] section in the Reference Guide.
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/replication.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/replication.adoc b/src/main/site/asciidoc/replication.adoc
deleted file mode 100644
index 9089754..0000000
--- a/src/main/site/asciidoc/replication.adoc
+++ /dev/null
@@ -1,22 +0,0 @@
-////
-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.
-////
-
-= Apache HBase (TM) Replication
-
-This information has been moved to link:book.html#cluster_replication"[the Cluster Replication] section of the link:book.html[Apache HBase Reference Guide].

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/resources.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/resources.adoc b/src/main/site/asciidoc/resources.adoc
deleted file mode 100644
index fef217e..0000000
--- a/src/main/site/asciidoc/resources.adoc
+++ /dev/null
@@ -1,27 +0,0 @@
-////
-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.
-////
-= Other Apache HBase (TM) Resources
-
-== Books
-HBase: The Definitive Guide:: link:http://shop.oreilly.com/product/0636920014348.do[HBase: The Definitive Guide, _Random Access to Your Planet-Size Data_] by Lars George. Publisher: O'Reilly Media, Released: August 2011, Pages: 556.
-
-HBase In Action:: link:http://www.manning.com/dimidukkhurana[HBase In Action] By Nick Dimiduk and Amandeep Khurana.  Publisher: Manning, MEAP Began: January 2012, Softbound print: Fall 2012, Pages: 350.
-
-HBase Administration Cookbook:: link:http://www.packtpub.com/hbase-administration-for-optimum-database-performance-cookbook/book[HBase Administration Cookbook] by Yifeng Jiang.  Publisher: PACKT Publishing, Release: Expected August 2012, Pages: 335.
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/asciidoc/sponsors.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/sponsors.adoc b/src/main/site/asciidoc/sponsors.adoc
deleted file mode 100644
index 4d7ebf3..0000000
--- a/src/main/site/asciidoc/sponsors.adoc
+++ /dev/null
@@ -1,36 +0,0 @@
-////
-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.
-////
-
-= Apache HBase(TM) Sponsors
-
-First off, thanks to link:http://www.apache.org/foundation/thanks.html[all who sponsor] our parent, the Apache Software Foundation.
-
-The below companies have been gracious enough to provide their commerical tool offerings free of charge to the Apache HBase(TM) project.
-
-* The crew at link:http://www.ej-technologies.com/[ej-technologies] have been letting us use link:http://www.ej-technologies.com/products/jprofiler/overview.html[JProfiler] for years now. 
-
-* The lads at link:http://headwaysoftware.com/[headway software] have given us a license for link:http://headwaysoftware.com/products/?code=Restructure101[Restructure101] so we can untangle our interdependency mess.
-
-* link:http://www.yourkit.com[YourKit] allows us to use their link:http://www.yourkit.com/overview/index.jsp[Java Profiler].
-* Some of us use link:http://www.jetbrains.com/idea[IntelliJ IDEA] thanks to link:http://www.jetbrains.com/[JetBrains].
-* Thank you to Boris at link:http://www.vectorportal.com/[Vector Portal] for granting us a license on the image on which our logo is based.
-
-== Sponsoring the Apache Software Foundation">
-To contribute to the Apache Software Foundation, a good idea in our opinion, see the link:http://www.apache.org/foundation/sponsorship.html[ASF Sponsorship] page.
-


[07/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbase_replication_diagram.jpg
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbase_replication_diagram.jpg b/src/site/resources/images/hbase_replication_diagram.jpg
new file mode 100644
index 0000000..c110309
Binary files /dev/null and b/src/site/resources/images/hbase_replication_diagram.jpg differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbasecon2015.30percent.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbasecon2015.30percent.png b/src/site/resources/images/hbasecon2015.30percent.png
new file mode 100644
index 0000000..26896a4
Binary files /dev/null and b/src/site/resources/images/hbasecon2015.30percent.png differ

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbasecon2017.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbasecon2017.png b/src/site/resources/images/hbasecon2017.png
new file mode 100644
index 0000000..4b25f89
Binary files /dev/null and b/src/site/resources/images/hbasecon2017.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hbaseconasia2017.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hbaseconasia2017.png b/src/site/resources/images/hbaseconasia2017.png
new file mode 100644
index 0000000..8548870
Binary files /dev/null and b/src/site/resources/images/hbaseconasia2017.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hfile.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hfile.png b/src/site/resources/images/hfile.png
new file mode 100644
index 0000000..5762970
Binary files /dev/null and b/src/site/resources/images/hfile.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/hfilev2.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/hfilev2.png b/src/site/resources/images/hfilev2.png
new file mode 100644
index 0000000..54cc0cf
Binary files /dev/null and b/src/site/resources/images/hfilev2.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/jumping-orca_rotated.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/jumping-orca_rotated.png b/src/site/resources/images/jumping-orca_rotated.png
new file mode 100644
index 0000000..4c2c72e
Binary files /dev/null and b/src/site/resources/images/jumping-orca_rotated.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/jumping-orca_rotated.xcf
----------------------------------------------------------------------
diff --git a/src/site/resources/images/jumping-orca_rotated.xcf b/src/site/resources/images/jumping-orca_rotated.xcf
new file mode 100644
index 0000000..01be6ff
Binary files /dev/null and b/src/site/resources/images/jumping-orca_rotated.xcf differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/jumping-orca_rotated_12percent.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/jumping-orca_rotated_12percent.png b/src/site/resources/images/jumping-orca_rotated_12percent.png
new file mode 100644
index 0000000..1942f9a
Binary files /dev/null and b/src/site/resources/images/jumping-orca_rotated_12percent.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/jumping-orca_rotated_25percent.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/jumping-orca_rotated_25percent.png b/src/site/resources/images/jumping-orca_rotated_25percent.png
new file mode 100644
index 0000000..219c657
Binary files /dev/null and b/src/site/resources/images/jumping-orca_rotated_25percent.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/jumping-orca_transparent_rotated.xcf
----------------------------------------------------------------------
diff --git a/src/site/resources/images/jumping-orca_transparent_rotated.xcf b/src/site/resources/images/jumping-orca_transparent_rotated.xcf
new file mode 100644
index 0000000..be9e3d9
Binary files /dev/null and b/src/site/resources/images/jumping-orca_transparent_rotated.xcf differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/region_split_process.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/region_split_process.png b/src/site/resources/images/region_split_process.png
new file mode 100644
index 0000000..2717617
Binary files /dev/null and b/src/site/resources/images/region_split_process.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/region_states.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/region_states.png b/src/site/resources/images/region_states.png
new file mode 100644
index 0000000..ba69e97
Binary files /dev/null and b/src/site/resources/images/region_states.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/replication_overview.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/replication_overview.png b/src/site/resources/images/replication_overview.png
new file mode 100644
index 0000000..47d7b4c
Binary files /dev/null and b/src/site/resources/images/replication_overview.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/images/timeline_consistency.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/timeline_consistency.png b/src/site/resources/images/timeline_consistency.png
new file mode 100644
index 0000000..94c47e0
Binary files /dev/null and b/src/site/resources/images/timeline_consistency.png differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar
new file mode 100644
index 0000000..5b93209
Binary files /dev/null and b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.jar differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
new file mode 100644
index 0000000..d12092b
--- /dev/null
+++ b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/1.5-HBASE/maven-fluido-skin-1.5-HBASE.pom
@@ -0,0 +1,718 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.maven.skins</groupId>
+    <artifactId>maven-skins</artifactId>
+    <version>10</version>
+    <relativePath>../maven-skins/pom.xml</relativePath>
+  </parent>
+
+  <artifactId>maven-fluido-skin</artifactId>
+  <version>1.5-HBASE</version>
+
+  <name>Apache Maven Fluido Skin</name>
+  <description>The Apache Maven Fluido Skin is an Apache Maven site skin
+    built on top of Twitter's bootstrap.</description>
+  <inceptionYear>2011</inceptionYear>
+
+  <scm>
+    <connection>scm:svn:http://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</connection>
+    <developerConnection>scm:svn:https://svn.apache.org/repos/asf/maven/skins/trunk/maven-fluido-skin/</developerConnection>
+    <url>http://svn.apache.org/viewvc/maven/skins/trunk/maven-fluido-skin/</url>
+  </scm>
+  <issueManagement>
+    <system>jira</system>
+    <url>https://issues.apache.org/jira/browse/MSKINS/component/12326474</url>
+  </issueManagement>
+  <distributionManagement>
+    <site>
+      <id>apache.website</id>
+      <url>scm:svn:https://svn.apache.org/repos/infra/websites/production/maven/components/${maven.site.path}</url>
+    </site>
+  </distributionManagement>
+
+  <contributors>
+    <!-- in alphabetical order -->
+    <contributor>
+      <name>Bruno P. Kinoshita</name>
+      <email>brunodepaulak AT yahoo DOT com DOT br</email>
+    </contributor>
+    <contributor>
+      <name>Carlos Villaronga</name>
+      <email>cvillaronga AT gmail DOT com</email>
+    </contributor>
+    <contributor>
+      <name>Christian Grobmeier</name>
+      <email>grobmeier AT apache DOT org</email>
+    </contributor>
+    <contributor>
+      <name>Conny Kreyssel</name>
+      <email>dev AT kreyssel DOT org</email>
+    </contributor>
+    <contributor>
+      <name>Michael Koch</name>
+      <email>tensberg AT gmx DOT net</email>
+    </contributor>
+    <contributor>
+      <name>Emmanuel Hugonnet</name>
+      <email>emmanuel DOT hugonnet AT gmail DOT com</email>
+    </contributor>
+    <contributor>
+      <name>Ivan Habunek</name>
+      <email>ihabunek AT apache DOT org</email>
+    </contributor>
+    <contributor>
+      <name>Eric Barboni</name>
+    </contributor>
+    <contributor>
+      <name>Michael Osipov</name>
+      <email>michaelo AT apache DOT org</email>
+    </contributor>
+  </contributors>
+
+  <properties>
+    <bootstrap.version>2.3.2</bootstrap.version>
+    <jquery.version>1.11.2</jquery.version>
+  </properties>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>.</directory>
+        <targetPath>META-INF</targetPath>
+        <includes>
+          <include>NOTICE</include>
+          <include>LICENSE</include>
+        </includes>
+      </resource>
+
+      <!-- exclude css and js since will include the minified version -->
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <excludes>
+          <exclude>css/**</exclude>
+          <exclude>js/**</exclude>
+        </excludes>
+        <filtering>true</filtering> <!-- add skin-info -->
+      </resource>
+
+      <!-- include the print.css -->
+      <resource>
+        <directory>${basedir}/src/main/resources</directory>
+        <includes>
+          <include>css/print.css</include>
+        </includes>
+      </resource>
+
+      <!-- include minified only -->
+      <resource>
+        <directory>${project.build.directory}/${project.build.finalName}</directory>
+        <includes>
+          <include>css/apache-maven-fluido-${project.version}.min.css</include>
+          <include>js/apache-maven-fluido-${project.version}.min.js</include>
+        </includes>
+      </resource>
+    </resources>
+
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.apache.rat</groupId>
+          <artifactId>apache-rat-plugin</artifactId>
+          <configuration>
+            <excludes combine.children="append">
+              <exclude>src/main/resources/fonts/glyphicons-halflings-regular.svg</exclude>
+              <exclude>src/main/resources/js/prettify.js</exclude>
+              <exclude>src/main/resources/js/jquery-*.js</exclude>
+            </excludes>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-resources-plugin</artifactId>
+        <dependencies><!-- TODO remove when upgrading to version 2.8: see MSHARED-325 / MRESOURCES-192 -->
+          <dependency>
+              <groupId>org.apache.maven.shared</groupId>
+              <artifactId>maven-filtering</artifactId>
+              <version>1.3</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <delimiters>
+            <delimiter>@</delimiter>
+          </delimiters>
+          <useDefaultDelimiters>false</useDefaultDelimiters>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>com.samaxes.maven</groupId>
+        <artifactId>maven-minify-plugin</artifactId>
+        <version>1.3.5</version>
+        <executions>
+          <execution>
+            <id>default-minify</id>
+            <phase>generate-resources</phase>
+            <configuration>
+              <webappSourceDir>${basedir}/src/main/resources</webappSourceDir>
+              <cssSourceDir>css</cssSourceDir>
+              <cssSourceFiles>
+                <cssSourceFile>bootstrap-${bootstrap.version}.css</cssSourceFile>
+                <cssSourceFile>maven-base.css</cssSourceFile>
+                <cssSourceFile>maven-theme.css</cssSourceFile>
+                <cssSourceFile>prettify.css</cssSourceFile>
+              </cssSourceFiles>
+              <cssFinalFile>apache-maven-fluido-${project.version}.css</cssFinalFile>
+              <jsSourceDir>js</jsSourceDir>
+              <jsSourceFiles>
+                <jsSourceFile>jquery-${jquery.version}.js</jsSourceFile>
+                <jsSourceFile>bootstrap-${bootstrap.version}.js</jsSourceFile>
+                <jsSourceFile>prettify.js</jsSourceFile>
+                <jsSourceFile>fluido.js</jsSourceFile>
+              </jsSourceFiles>
+              <jsFinalFile>apache-maven-fluido-${project.version}.js</jsFinalFile>
+            </configuration>
+            <goals>
+              <goal>minify</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>run-its</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-invoker-plugin</artifactId>
+            <configuration>
+              <debug>true</debug>
+              <projectsDirectory>src/it</projectsDirectory>
+              <cloneProjectsTo>${project.build.directory}/it</cloneProjectsTo>
+              <preBuildHookScript>setup</preBuildHookScript>
+              <postBuildHookScript>verify</postBuildHookScript>
+              <localRepositoryPath>${project.build.directory}/local-repo</localRepositoryPath>
+              <settingsFile>src/it/settings.xml</settingsFile>
+              <pomIncludes>
+                <pomInclude>*/pom.xml</pomInclude>
+              </pomIncludes>
+              <goals>
+                <goal>site</goal>
+              </goals>
+            </configuration>
+            <executions>
+              <execution>
+                <id>integration-test</id>
+                <goals>
+                  <goal>install</goal>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>reporting</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-resources-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>copy-sidebar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/sidebar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/sidebar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-topbar-inverse</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/topbar-inverse/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/topbar-inverse/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-10</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-10/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-10/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-13</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-13/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-13/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-14</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-14/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-14/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-14_sitesearch</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-14_sitesearch/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-14_sitesearch/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-15</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-15/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-15/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-16</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-16/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-16/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-17</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-17/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-17/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-21</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-21/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-21/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-22</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-22/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-22/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-22_default</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-22_default/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-22_default/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-22_topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-22_topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-22_topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-23</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-23/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-23/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-24</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-24/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-24/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-24_topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-24_topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-24_topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-25</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-25/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-25/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-28</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-28/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-28/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-31</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-31/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-31/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-33</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-33/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-33/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-33_topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-33_topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-33_topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-34</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-34/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-34/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-34_topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-34_topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-34_topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-41</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-41/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-41/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-72</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-72/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-72/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-75</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-75/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-75/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-76</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-76/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-76/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-76_topbar</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-76_topbar/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-76_topbar/</outputDirectory>
+                </configuration>
+              </execution>
+              <execution>
+                <id>copy-mskins-85</id>
+                <phase>site</phase>
+                <goals>
+                  <goal>copy-resources</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>${project.build.directory}/it/mskins-85/target/site/</directory>
+                    </resource>
+                  </resources>
+                  <outputDirectory>${project.build.directory}/site/mskins-85/</outputDirectory>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+      <reporting>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-invoker-plugin</artifactId>
+            <version>1.8</version>
+          </plugin>
+        </plugins>
+      </reporting>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
----------------------------------------------------------------------
diff --git a/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
new file mode 100644
index 0000000..65791e8
--- /dev/null
+++ b/src/site/resources/repo/org/apache/maven/skins/maven-fluido-skin/maven-metadata-local.xml
@@ -0,0 +1,12 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<metadata>
+  <groupId>org.apache.maven.skins</groupId>
+  <artifactId>maven-fluido-skin</artifactId>
+  <versioning>
+    <release>1.5-HBASE</release>
+    <versions>
+      <version>1.5-HBASE</version>
+    </versions>
+    <lastUpdated>20151111033340</lastUpdated>
+  </versioning>
+</metadata>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
new file mode 100644
index 0000000..e038f91
--- /dev/null
+++ b/src/site/site.xml
@@ -0,0 +1,131 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<project xmlns="http://maven.apache.org/DECORATION/1.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/DECORATION/1.0.0 http://maven.apache.org/xsd/decoration-1.0.0.xsd">
+  <skin>
+    <groupId>org.apache.maven.skins</groupId>
+    <artifactId>maven-fluido-skin</artifactId>
+    <version>1.5-HBASE</version>
+  </skin>
+  <custom>
+    <fluidoSkin>
+      <topBarEnabled>true</topBarEnabled>
+      <sideBarEnabled>false</sideBarEnabled>
+      <googleSearch>
+        <!-- The ID of the Google custom search engine to use.
+             This one searches hbase.apache.org, issues.apache.org/browse/HBASE-*,
+             and user and dev mailing list archives. -->
+        <customSearch>000385458301414556862:sq1bb0xugjg</customSearch>
+      </googleSearch>
+      <sourceLineNumbersEnabled>false</sourceLineNumbersEnabled>
+      <skipGenerationDate>true</skipGenerationDate>
+      <breadcrumbDivider>»</breadcrumbDivider>
+    </fluidoSkin>
+  </custom>
+  <bannerLeft>
+    <name>HBaseCon2017</name>
+    <src>images/hbaseconasia2017.png</src>
+    <href>https://www.eventbrite.com/e/hbasecon-asia-2017-tickets-34935546159</href>
+    <!--
+    <name/>
+    <height>0</height>
+    <width>0</width>
+-->
+  </bannerLeft>
+  <bannerRight>
+    <name>Apache HBase</name>
+    <src>images/hbase_logo_with_orca_large.png</src>
+    <href>http://hbase.apache.org/</href>
+  </bannerRight>
+  <publishDate position="bottom"/>
+  <version position="none"/>
+  <body>
+    <head>
+      <meta name="viewport" content="width=device-width, initial-scale=1.0"></meta>
+      <link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/twitter-bootstrap/2.3.2/css/bootstrap-responsive.min.css"/>
+      <link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.9.1/styles/github.min.css"/>
+      <link rel="stylesheet" href="css/site.css"/>
+      <script src="https://cdnjs.cloudflare.com/ajax/libs/highlight.js/8.9.1/highlight.min.js"></script>
+    </head>
+    <menu name="Apache HBase Project">
+      <item name="Overview" href="index.html"/>
+      <item name="License" href="license.html"/>
+      <item name="Downloads" href="http://www.apache.org/dyn/closer.cgi/hbase/"/>
+      <item name="Release Notes" href="https://issues.apache.org/jira/browse/HBASE?report=com.atlassian.jira.plugin.system.project:changelog-panel#selectedTab=com.atlassian.jira.plugin.system.project%3Achangelog-panel" />
+      <item name="Code Of Conduct" href="coc.html"/>
+      <item name="Blog" href="http://blogs.apache.org/hbase/"/>
+      <item name="Mailing Lists" href="mail-lists.html"/>
+      <item name="Team" href="team-list.html"/>
+      <item name="ReviewBoard" href="https://reviews.apache.org/"/>
+      <item name="Thanks" href="sponsors.html"/>
+      <item name="Powered by HBase" href="poweredbyhbase.html"/>
+      <item name="Other resources" href="resources.html"/>
+    </menu>
+    <menu name="Project Information">
+      <item name="Project Summary" href="project-summary.html"/>
+      <item name="Dependency Information" href="dependency-info.html"/>
+      <item name="Team" href="team-list.html"/>
+      <item name="Source Repository" href="source-repository.html"/>
+      <item name="Issue Tracking" href="issue-tracking.html"/>
+      <item name="Dependency Management" href="dependency-management.html"/>
+      <item name="Dependencies" href="dependencies.html"/>
+      <item name="Dependency Convergence" href="dependency-convergence.html"/>
+      <item name="Continuous Integration" href="integration.html"/>
+      <item name="Plugin Management" href="plugin-management.html"/>
+      <item name="Plugins" href="plugins.html"/>
+    </menu>
+    <menu name="Documentation and API">
+      <item name="Reference Guide" href="book.html" target="_blank" />
+      <item name="Reference Guide (PDF)" href="apache_hbase_reference_guide.pdf" target="_blank" />
+      <item name="Getting Started" href="book.html#quickstart" target="_blank" />
+      <item name="User API" href="apidocs/index.html" target="_blank" />
+      <item name="User API (Test)" href="testapidocs/index.html" target="_blank" />
+      <item name="Developer API" href="devapidocs/index.html" target="_blank" />
+      <item name="Developer API (Test)" href="testdevapidocs/index.html" target="_blank" />
+      <item name="中文参考指南(单页)" href="http://abloz.com/hbase/book.html" target="_blank" />
+      <item name="FAQ" href="book.html#faq" target="_blank" />
+      <item name="Videos/Presentations" href="book.html#other.info" target="_blank" />
+      <item name="Wiki" href="http://wiki.apache.org/hadoop/Hbase" target="_blank" />
+      <item name="ACID Semantics" href="acid-semantics.html" target="_blank" />
+      <item name="Bulk Loads" href="book.html#arch.bulk.load" target="_blank" />
+      <item name="Metrics" href="metrics.html" target="_blank" />
+      <item name="HBase on Windows" href="cygwin.html" target="_blank" />
+      <item name="Cluster replication" href="book.html#replication" target="_blank" />
+      <item name="1.2 Documentation">
+        <item name="API" href="1.2/apidocs/index.html" target="_blank" />
+        <item name="X-Ref" href="1.2/xref/index.html" target="_blank" />
+        <item name="Ref Guide (single-page)" href="1.2/book.html" target="_blank" />
+      </item>
+      <item name="1.1 Documentation">
+        <item name="API" href="1.1/apidocs/index.html" target="_blank" />
+        <item name="X-Ref" href="1.1/xref/index.html" target="_blank" />
+        <item name="Ref Guide (single-page)" href="1.1/book.html" target="_blank" />
+      </item>
+    </menu>
+    <menu name="ASF">
+      <item name="Apache Software Foundation" href="http://www.apache.org/foundation/" target="_blank" />
+      <item name="How Apache Works" href="http://www.apache.org/foundation/how-it-works.html" target="_blank" />
+      <item name="Sponsoring Apache" href="http://www.apache.org/foundation/sponsorship.html" target="_blank" />
+    </menu>
+    </body>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/acid-semantics.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/acid-semantics.xml b/src/site/xdoc/acid-semantics.xml
new file mode 100644
index 0000000..2d4eb6a
--- /dev/null
+++ b/src/site/xdoc/acid-semantics.xml
@@ -0,0 +1,235 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title> 
+      Apache HBase (TM) ACID Properties
+    </title>
+  </properties>
+
+  <body>
+    <section name="About this Document">
+      <p>Apache HBase (TM) is not an ACID compliant database. However, it does guarantee certain specific
+      properties.</p>
+      <p>This specification enumerates the ACID properties of HBase.</p>
+    </section>
+    <section name="Definitions">
+      <p>For the sake of common vocabulary, we define the following terms:</p>
+      <dl>
+        <dt>Atomicity</dt>
+        <dd>an operation is atomic if it either completes entirely or not at all</dd>
+
+        <dt>Consistency</dt>
+        <dd>
+          all actions cause the table to transition from one valid state directly to another
+          (eg a row will not disappear during an update, etc)
+        </dd>
+
+        <dt>Isolation</dt>
+        <dd>
+          an operation is isolated if it appears to complete independently of any other concurrent transaction
+        </dd>
+
+        <dt>Durability</dt>
+        <dd>any update that reports &quot;successful&quot; to the client will not be lost</dd>
+
+        <dt>Visibility</dt>
+        <dd>an update is considered visible if any subsequent read will see the update as having been committed</dd>
+      </dl>
+      <p>
+        The terms <em>must</em> and <em>may</em> are used as specified by RFC 2119.
+        In short, the word &quot;must&quot; implies that, if some case exists where the statement
+        is not true, it is a bug. The word &quot;may&quot; implies that, even if the guarantee
+        is provided in a current release, users should not rely on it.
+      </p>
+    </section>
+    <section name="APIs to consider">
+      <ul>
+        <li>Read APIs
+        <ul>
+          <li>get</li>
+          <li>scan</li>
+        </ul>
+        </li>
+        <li>Write APIs</li>
+        <ul>
+          <li>put</li>
+          <li>batch put</li>
+          <li>delete</li>
+        </ul>
+        <li>Combination (read-modify-write) APIs</li>
+        <ul>
+          <li>incrementColumnValue</li>
+          <li>checkAndPut</li>
+        </ul>
+      </ul>
+    </section>
+
+    <section name="Guarantees Provided">
+
+      <section name="Atomicity">
+
+        <ol>
+          <li>All mutations are atomic within a row. Any put will either wholly succeed or wholly fail.[3]</li>
+          <ol>
+            <li>An operation that returns a &quot;success&quot; code has completely succeeded.</li>
+            <li>An operation that returns a &quot;failure&quot; code has completely failed.</li>
+            <li>An operation that times out may have succeeded and may have failed. However,
+            it will not have partially succeeded or failed.</li>
+          </ol>
+          <li> This is true even if the mutation crosses multiple column families within a row.</li>
+          <li> APIs that mutate several rows will _not_ be atomic across the multiple rows.
+          For example, a multiput that operates on rows 'a','b', and 'c' may return having
+          mutated some but not all of the rows. In such cases, these APIs will return a list
+          of success codes, each of which may be succeeded, failed, or timed out as described above.</li>
+          <li> The checkAndPut API happens atomically like the typical compareAndSet (CAS) operation
+          found in many hardware architectures.</li>
+          <li> The order of mutations is seen to happen in a well-defined order for each row, with no
+          interleaving. For example, if one writer issues the mutation &quot;a=1,b=1,c=1&quot; and
+          another writer issues the mutation &quot;a=2,b=2,c=2&quot;, the row must either
+          be &quot;a=1,b=1,c=1&quot; or &quot;a=2,b=2,c=2&quot; and must <em>not</em> be something
+          like &quot;a=1,b=2,c=1&quot;.</li>
+          <ol>
+            <li>Please note that this is not true _across rows_ for multirow batch mutations.</li>
+          </ol>
+        </ol>
+      </section>
+      <section name="Consistency and Isolation">
+        <ol>
+          <li>All rows returned via any access API will consist of a complete row that existed at
+          some point in the table's history.</li>
+          <li>This is true across column families - i.e a get of a full row that occurs concurrent
+          with some mutations 1,2,3,4,5 will return a complete row that existed at some point in time
+          between mutation i and i+1 for some i between 1 and 5.</li>
+          <li>The state of a row will only move forward through the history of edits to it.</li>
+        </ol>
+
+        <section name="Consistency of Scans">
+        <p>
+          A scan is <strong>not</strong> a consistent view of a table. Scans do
+          <strong>not</strong> exhibit <em>snapshot isolation</em>.
+        </p>
+        <p>
+          Rather, scans have the following properties:
+        </p>
+
+        <ol>
+          <li>
+            Any row returned by the scan will be a consistent view (i.e. that version
+            of the complete row existed at some point in time) [1]
+          </li>
+          <li>
+            A scan will always reflect a view of the data <em>at least as new as</em>
+            the beginning of the scan. This satisfies the visibility guarantees
+          enumerated below.</li>
+          <ol>
+            <li>For example, if client A writes data X and then communicates via a side
+            channel to client B, any scans started by client B will contain data at least
+            as new as X.</li>
+            <li>A scan _must_ reflect all mutations committed prior to the construction
+            of the scanner, and _may_ reflect some mutations committed subsequent to the
+            construction of the scanner.</li>
+            <li>Scans must include <em>all</em> data written prior to the scan (except in
+            the case where data is subsequently mutated, in which case it _may_ reflect
+            the mutation)</li>
+          </ol>
+        </ol>
+        <p>
+          Those familiar with relational databases will recognize this isolation level as &quot;read committed&quot;.
+        </p>
+        <p>
+          Please note that the guarantees listed above regarding scanner consistency
+          are referring to &quot;transaction commit time&quot;, not the &quot;timestamp&quot;
+          field of each cell. That is to say, a scanner started at time <em>t</em> may see edits
+          with a timestamp value greater than <em>t</em>, if those edits were committed with a
+          &quot;forward dated&quot; timestamp before the scanner was constructed.
+        </p>
+        </section>
+      </section>
+      <section name="Visibility">
+        <ol>
+          <li> When a client receives a &quot;success&quot; response for any mutation, that
+          mutation is immediately visible to both that client and any client with whom it
+          later communicates through side channels. [3]</li>
+          <li> A row must never exhibit so-called &quot;time-travel&quot; properties. That
+          is to say, if a series of mutations moves a row sequentially through a series of
+          states, any sequence of concurrent reads will return a subsequence of those states.</li>
+          <ol>
+            <li>For example, if a row's cells are mutated using the &quot;incrementColumnValue&quot;
+            API, a client must never see the value of any cell decrease.</li>
+            <li>This is true regardless of which read API is used to read back the mutation.</li>
+          </ol>
+          <li> Any version of a cell that has been returned to a read operation is guaranteed to
+          be durably stored.</li>
+        </ol>
+
+      </section>
+      <section name="Durability">
+        <ol>
+          <li> All visible data is also durable data. That is to say, a read will never return
+          data that has not been made durable on disk[2]</li>
+          <li> Any operation that returns a &quot;success&quot; code (eg does not throw an exception)
+          will be made durable.[3]</li>
+          <li> Any operation that returns a &quot;failure&quot; code will not be made durable
+          (subject to the Atomicity guarantees above)</li>
+          <li> All reasonable failure scenarios will not affect any of the guarantees of this document.</li>
+
+        </ol>
+      </section>
+      <section name="Tunability">
+        <p>All of the above guarantees must be possible within Apache HBase. For users who would like to trade
+        off some guarantees for performance, HBase may offer several tuning options. For example:</p>
+        <ul>
+          <li>Visibility may be tuned on a per-read basis to allow stale reads or time travel.</li>
+          <li>Durability may be tuned to only flush data to disk on a periodic basis</li>
+        </ul>
+      </section>
+    </section>
+    <section name="More Information">
+      <p>
+      For more information, see the <a href="book.html#client">client architecture</a> or <a href="book.html#datamodel">data model</a> sections in the Apache HBase Reference Guide. 
+      </p>
+    </section>
+    
+    <section name="Footnotes">
+      <p>[1] A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of
+          a row in one RPC then going back to fetch another portion of the row in a subsequent RPC.
+          Intra-row scanning happens when you set a limit on how many values to return per Scan#next
+          (See <a href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)">Scan#setBatch(int)</a>).
+      </p>
+
+      <p>[2] In the context of Apache HBase, &quot;durably on disk&quot; implies an hflush() call on the transaction
+      log. This does not actually imply an fsync() to magnetic media, but rather just that the data has been
+      written to the OS cache on all replicas of the log. In the case of a full datacenter power loss, it is
+      possible that the edits are not truly durable.</p>
+      <p>[3] Puts will either wholly succeed or wholly fail, provided that they are actually sent
+      to the RegionServer.  If the writebuffer is used, Puts will not be sent until the writebuffer is filled
+      or it is explicitly flushed.</p>
+      
+    </section>
+
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/bulk-loads.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/bulk-loads.xml b/src/site/xdoc/bulk-loads.xml
new file mode 100644
index 0000000..2195003
--- /dev/null
+++ b/src/site/xdoc/bulk-loads.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title> 
+      Bulk Loads in Apache HBase (TM)
+    </title>
+  </properties>
+  <body>
+       <p>This page has been retired.  The contents have been moved to the 
+      <a href="http://hbase.apache.org/book.html#arch.bulk.load">Bulk Loading</a> section
+ in the Reference Guide.
+ </p>
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/coc.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/coc.xml b/src/site/xdoc/coc.xml
new file mode 100644
index 0000000..fc2b549
--- /dev/null
+++ b/src/site/xdoc/coc.xml
@@ -0,0 +1,92 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>
+      Code of Conduct Policy
+    </title>
+  </properties>
+  <body>
+  <section name="Code of Conduct Policy">
+<p>
+We expect participants in discussions on the HBase project mailing lists, IRC
+channels, and JIRA issues to abide by the Apache Software Foundation's
+<a href="http://apache.org/foundation/policies/conduct.html">Code of Conduct</a>.
+</p>
+<p>
+If you feel there has been a violation of this code, please point out your
+concerns publicly in a friendly and matter of fact manner. Nonverbal
+communication is prone to misinterpretation and misunderstanding. Everyone has
+bad days and sometimes says things they regret later. Someone else's
+communication style may clash with yours, but the difference can be amicably
+resolved. After pointing out your concerns please be generous upon receiving an
+apology.
+</p>
+<p>
+Should there be repeated instances of code of conduct violations, or if there is
+an obvious and severe violation, the HBase PMC may become involved. When this
+happens the PMC will openly discuss the matter, most likely on the dev@hbase
+mailing list, and will consider taking the following actions, in order, if there
+is a continuing problem with an individual:
+<ol>
+<li>A friendly off-list warning;</li>
+<li>A friendly public warning, if the communication at issue was on list, otherwise another off-list warning;</li>
+<li>A three month suspension from the public mailing lists and possible operator action in the IRC channels.</li>
+<li>A permanent ban from the public mailing lists, IRC channels, and project JIRA.</li>
+</ol>
+</p>
+<p>
+For flagrant violations requiring a firm response the PMC may opt to skip early
+steps. No action will be taken before public discussion leading to consensus or
+a successful majority vote.
+</p>
+  </section>
+  <section name="Diversity Statement">
+<p>
+As a project and a community, we encourage you to participate in the HBase project
+in whatever capacity suits you, whether it involves development, documentation,
+answering questions on mailing lists, triaging issue and patch review, managing
+releases, or any other way that you want to help. We appreciate your
+contributions and the time you dedicate to the HBase project. We strive to
+recognize the work of participants publicly. Please let us know if we can
+improve in this area.
+</p>
+<p>
+We value diversity and strive to support participation by people with all
+different backgrounds. Rich projects grow from groups with different points of
+view and different backgrounds. We welcome your suggestions about how we can
+welcome participation by people at all skill levels and with all aspects of the
+project.
+</p>
+<p>
+If you can think of something we are doing that we shouldn't, or something that
+we should do but aren't, please let us know. If you feel comfortable doing so,
+use the public mailing lists. Otherwise, reach out to a PMC member or send an
+email to <a href="mailto:private@hbase.apache.org">the private PMC mailing list</a>.
+</p>
+  </section>
+  </body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/cygwin.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/cygwin.xml b/src/site/xdoc/cygwin.xml
new file mode 100644
index 0000000..406c0a9
--- /dev/null
+++ b/src/site/xdoc/cygwin.xml
@@ -0,0 +1,245 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>Installing Apache HBase (TM) on Windows using Cygwin</title>
+  </properties>
+
+<body>
+<section name="Introduction">
+<p><a title="HBase project" href="http://hbase.apache.org" target="_blank">Apache HBase (TM)</a> is a distributed, column-oriented store, modeled after Google's <a title="Google's BigTable" href="http://research.google.com/archive/bigtable.html" target="_blank">BigTable</a>. Apache HBase is built on top of <a title="Hadoop project" href="http://hadoop.apache.org">Hadoop</a> for its <a title="Hadoop MapReduce project" href="http://hadoop.apache.org/mapreduce" target="_blank">MapReduce </a>and <a title="Hadoop DFS project" href="http://hadoop.apache.org/hdfs">distributed file system</a> implementation. All these projects are open-source and part of the <a title="The Apache Software Foundation" href="http://www.apache.org/" target="_blank">Apache Software Foundation</a>.</p>
+
+<p style="text-align: justify; ">As being distributed, large scale platforms, the Hadoop and HBase projects mainly focus on <em><strong>*nix</strong></em><strong> environments</strong> for production installations. However, being developed in <strong>Java</strong>, both projects are fully <strong>portable</strong> across platforms and, hence, also to the <strong>Windows operating system</strong>. For ease of development the projects rely on <a title="Cygwin site" href="http://www.cygwin.com/" target="_blank">Cygwin</a> to have a *nix-like environment on Windows to run the shell scripts.</p>
+</section>
+<section name="Purpose">
+<p style="text-align: justify; ">This document explains the <strong>intricacies of running Apache HBase on Windows using Cygwin</strong> as an all-in-one single-node installation for testing and development. The HBase <a title="HBase Overview" href="http://hbase.apache.org/apidocs/overview-summary.html#overview_description" target="_blank">Overview</a> and <a title="HBase QuickStart" href="http://hbase.apache.org/book/quickstart.html" target="_blank">QuickStart</a> guides on the other hand go a long way in explaning how to setup <a title="HBase project" href="http://hadoop.apache.org/hbase" target="_blank">HBase</a> in more complex deployment scenario's.</p>
+</section>
+
+<section name="Installation">
+<p style="text-align: justify; ">For running Apache HBase on Windows, 3 technologies are required: <strong>Java, Cygwin and SSH</strong>. The following paragraphs detail the installation of each of the aforementioned technologies.</p>
+<section name="Java">
+<p style="text-align: justify; ">HBase depends on the <a title="Java Platform, Standard Edition, 6 Release" href="http://java.sun.com/javase/6/" target="_blank">Java Platform, Standard Edition, 6 Release</a>. So the target system has to be provided with at least the Java Runtime Environment (JRE); however if the system will also be used for development, the Jave Development Kit (JDK) is preferred. You can download the latest versions for both from <a title="Java SE Downloads" href="http://java.sun.com/javase/downloads/index.jsp" target="_blank">Sun's download page</a>. Installation is a simple GUI wizard that guides you through the process.</p>
+</section>
+<section name="Cygwin">
+<p style="text-align: justify; ">Cygwin is probably the oddest technology in this solution stack. It provides a dynamic link library that emulates most of a *nix environment on Windows. On top of that a whole bunch of the most common *nix tools are supplied. Combined, the DLL with the tools form a very *nix-alike environment on Windows.</p>
+
+<p style="text-align: justify; ">For installation, Cygwin provides the <a title="Cygwin Setup Utility" href="http://cygwin.com/setup.exe" target="_blank"><strong><code>setup.exe</code> utility</strong></a> that tracks the versions of all installed components on the target system and provides the mechanism for <strong>installing</strong> or <strong>updating </strong>everything from the mirror sites of Cygwin.</p>
+
+<p style="text-align: justify; ">To support installation, the <code>setup.exe</code> utility uses 2 directories on the target system. The <strong>Root</strong> directory for Cygwin (defaults to <code>C:\cygwin)</code> which will become <code>/</code> within the eventual Cygwin installation; and the <strong>Local Package </strong>directory (e.g. <code>C:\cygsetup</code> that is the cache where <code>setup.exe</code> stores the packages before they are installed. The cache must not be the same folder as the Cygwin root.</p>
+
+<p style="text-align: justify; ">Perform following steps to install Cygwin, which are elaboratly detailed in the <a title="Setting Up Cygwin" href="http://cygwin.com/cygwin-ug-net/setup-net.html" target="_self">2nd chapter</a> of the <a title="Cygwin User's Guide" href="http://cygwin.com/cygwin-ug-net/cygwin-ug-net.html" target="_blank">Cygwin User's Guide</a>:</p>
+
+<ol style="text-align: justify; ">
+	<li>Make sure you have <code>Administrator</code> privileges on the target system.</li>
+	<li>Choose and create you <strong>Root</strong> and <strong>Local Package</strong> directories. A good suggestion is to use <code>C:\cygwin\root</code> and <code>C:\cygwin\setup</code> folders.</li>
+	<li>Download the <code>setup.exe</code> utility and save it to the <strong>Local Package</strong> directory.</li>
+	<li>Run the <code>setup.exe</code> utility,
+<ol>
+	<li>Choose  the <code>Install from Internet</code> option,</li>
+	<li>Choose your <strong>Root</strong> and <strong>Local Package</strong> folders</li>
+	<li>and select an appropriate mirror.</li>
+	<li>Don't select any additional packages yet, as we only want to install Cygwin for now.</li>
+	<li>Wait for download and install</li>
+	<li>Finish the installation</li>
+</ol>
+</li>
+	<li>Optionally, you can now also add a shortcut to your Start menu pointing to the <code>setup.exe</code> utility in the <strong>Local Package </strong>folder.</li>
+	<li>Add <code>CYGWIN_HOME</code> system-wide environment variable that points to your <strong>Root </strong>directory.</li>
+	<li>Add <code>%CYGWIN_HOME%\bin</code> to the end of your <code>PATH</code> environment variable.</li>
+	<li>Reboot the sytem after making changes to the environment variables otherwise the OS will not be able to find the Cygwin utilities.</li>
+	<li>Test your installation by running your freshly created shortcuts or the <code>Cygwin.bat</code> command in the <strong>Root</strong> folder. You should end up in a terminal window that is running a <a title="Bash Reference Manual" href="http://www.gnu.org/software/bash/manual/bashref.html" target="_blank">Bash shell</a>. Test the shell by issuing following commands:
+<ol>
+	<li><code>cd /</code> should take you to thr <strong>Root</strong> directory in Cygwin;</li>
+	<li>the <code>LS</code> commands that should list all files and folders in the current directory.</li>
+	<li>Use the <code>exit</code> command to end the terminal.</li>
+</ol>
+</li>
+	<li>When needed, to <strong>uninstall</strong> Cygwin you can simply delete the <strong>Root</strong> and <strong>Local Package</strong> directory, and the <strong>shortcuts</strong> that were created during installation.</li>
+</ol>
+</section>
+<section name="SSH">
+<p style="text-align: justify; ">HBase (and Hadoop) rely on <a title="Secure Shell" href="http://nl.wikipedia.org/wiki/Secure_Shell" target="_blank"><strong>SSH</strong></a> for interprocess/-node <strong>communication</strong> and launching<strong> remote commands</strong>. SSH will be provisioned on the target system via Cygwin, which supports running Cygwin programs as <strong>Windows services</strong>!</p>
+
+<ol style="text-align: justify; ">
+	<li>Rerun the <code><strong>setup.exe</strong></code><strong> utility</strong>.</li>
+	<li>Leave all parameters as is, skipping through the wizard using the <code>Next</code> button until the <code>Select Packages</code> panel is shown.</li>
+	<li>Maximize the window and click the <code>View</code> button to toggle to the list view, which is ordered alfabetically on <code>Package</code>, making it easier to find the packages we'll need.</li>
+	<li>Select the following packages by clicking the status word (normally <code>Skip</code>) so it's marked for installation. Use the <code>Next </code>button to download and install the packages.
+<ol>
+	<li>OpenSSH</li>
+	<li>tcp_wrappers</li>
+	<li>diffutils</li>
+	<li>zlib</li>
+</ol>
+</li>
+	<li>Wait for the install to complete and finish the installation.</li>
+</ol>
+</section>
+<section name="HBase">
+<p style="text-align: justify; ">Download the <strong>latest release </strong>of Apache HBase from the <a title="HBase Releases" href="http://www.apache.org/dyn/closer.cgi/hbase/" target="_blank">website</a>. As the Apache HBase distributable is just a zipped archive, installation is as simple as unpacking the archive so it ends up in its final <strong>installation</strong> directory. Notice that HBase has to be installed in Cygwin and a good directory suggestion is to use <code>/usr/local/</code> (or [<code><strong>Root</strong> directory]\usr\local</code> in Windows slang). You should end up with a <code>/usr/local/hbase-<em>&lt;version&gt;</em></code> installation in Cygwin.</p>
+
+This finishes installation. We go on with the configuration.
+</section>
+</section>
+<section name="Configuration">
+<p style="text-align: justify; ">There are 3 parts left to configure: <strong>Java, SSH and HBase</strong> itself. Following paragraphs explain eacht topic in detail.</p>
+<section name="Java">
+<p style="text-align: justify; ">One important thing to remember in shell scripting in general (i.e. *nix and Windows) is that managing, manipulating and assembling path names that contains spaces can be very hard, due to the need to escape and quote those characters and strings. So we try to stay away from spaces in path names. *nix environments can help us out here very easily by using <strong>symbolic links</strong>.</p>
+
+<ol style="text-align: justify; ">
+	<li style="text-align: justify; ">Create a link in <code>/usr/local</code> to the Java home directory by using the following command and substituting the name of your chosen Java environment:
+<pre>LN -s /cygdrive/c/Program\ Files/Java/<em>&lt;jre name&gt; </em>/usr/local/<em>&lt;jre name&gt;</em></pre>
+</li>
+	<li>Test your java installation by changing directories to your Java folder <code>CD /usr/local/<em>&lt;jre name&gt;</em></code> and issueing the command <code>./bin/java -version</code>. This should output your version of the chosen JRE.</li>
+</ol>
+</section>
+<section>
+<title>SSH</title>
+<p style="text-align: justify; ">Configuring <strong>SSH </strong>is quite elaborate, but primarily a question of launching it by default as a<strong> Windows service</strong>.</p>
+
+<ol style="text-align: justify; ">
+	<li style="text-align: justify; ">On Windows Vista and above make sure you run the Cygwin shell with <strong>elevated privileges</strong>, by right-clicking on the shortcut an using <code>Run as Administrator</code>.</li>
+	<li style="text-align: justify; ">First of all, we have to make sure the <strong>rights on some crucial files</strong> are correct. Use the commands underneath. You can verify all rights by using the <code>LS -L</code> command on the different files. Also, notice the auto-completion feature in the shell using <code>&lt;TAB&gt;</code> is extremely handy in these situations.
+<ol>
+	<li><code>chmod +r /etc/passwd</code> to make the passwords file readable for all</li>
+	<li><code>chmod u+w /etc/passwd</code> to make the passwords file writable for the owner</li>
+	<li><code>chmod +r /etc/group</code> to make the groups file readable for all</li>
+</ol>
+<ol>
+	<li><code>chmod u+w /etc/group</code> to make the groups file writable for the owner</li>
+</ol>
+<ol>
+	<li><code>chmod 755 /var</code> to make the var folder writable to owner and readable and executable to all</li>
+</ol>
+</li>
+	<li>Edit the <strong>/etc/hosts.allow</strong> file using your favorite editor (why not VI in the shell!) and make sure the following two lines are in there before the <code>PARANOID</code> line:
+<ol>
+	<li><code>ALL : localhost 127.0.0.1/32 : allow</code></li>
+	<li><code>ALL : [::1]/128 : allow</code></li>
+</ol>
+</li>
+	<li>Next we have to <strong>configure SSH</strong> by using the script <code>ssh-host-config</code>
+<ol>
+	<li>If this script asks to overwrite an existing <code>/etc/ssh_config</code>, answer <code>yes</code>.</li>
+	<li>If this script asks to overwrite an existing <code>/etc/sshd_config</code>, answer <code>yes</code>.</li>
+	<li>If this script asks to use privilege separation, answer <code>yes</code>.</li>
+	<li>If this script asks to install <code>sshd</code> as a service, answer <code>yes</code>. Make sure you started your shell as Adminstrator!</li>
+	<li>If this script asks for the CYGWIN value, just <code>&lt;enter&gt;</code> as the default is <code>ntsec</code>.</li>
+	<li>If this script asks to create the <code>sshd</code> account, answer <code>yes</code>.</li>
+	<li>If this script asks to use a different user name as service account, answer <code>no</code> as the default will suffice.</li>
+	<li>If this script asks to create the <code>cyg_server</code> account, answer <code>yes</code>. Enter a password for the account.</li>
+</ol>
+</li>
+	<li><strong>Start the SSH service</strong> using <code>net start sshd</code> or <code>cygrunsrv  --start  sshd</code>. Notice that <code>cygrunsrv</code> is the utility that make the process run as a Windows service. Confirm that you see a message stating that <code>the CYGWIN sshd service  was started succesfully.</code></li>
+	<li>Harmonize Windows and Cygwin<strong> user account</strong> by using the commands:
+<ol>
+	<li><code>mkpasswd -cl &gt; /etc/passwd</code></li>
+	<li><code>mkgroup --local &gt; /etc/group</code></li>
+</ol>
+</li>
+	<li><strong>Test </strong>the installation of SSH:
+<ol>
+	<li>Open a new Cygwin terminal</li>
+	<li>Use the command <code>whoami</code> to verify your userID</li>
+	<li>Issue an <code>ssh localhost</code> to connect to the system itself
+<ol>
+	<li>Answer <code>yes</code> when presented with the server's fingerprint</li>
+	<li>Issue your password when prompted</li>
+	<li>test a few commands in the remote session</li>
+	<li>The <code>exit</code> command should take you back to your first shell in Cygwin</li>
+</ol>
+</li>
+	<li><code>Exit</code> should terminate the Cygwin shell.</li>
+</ol>
+</li>
+</ol>
+</section>
+<section name="HBase">
+If all previous configurations are working properly, we just need some tinkering at the <strong>HBase config</strong> files to properly resolve on Windows/Cygwin. All files and paths referenced here start from the HBase <code>[<strong>installation</strong> directory]</code> as working directory.
+<ol>
+	<li>HBase uses the <code>./conf/<strong>hbase-env.sh</strong></code> to configure its dependencies on the runtime environment. Copy and uncomment following lines just underneath their original, change them to fit your environemnt. They should read something like:
+<ol>
+	<li><code>export JAVA_HOME=/usr/local/<em>&lt;jre name&gt;</em></code></li>
+	<li><code>export HBASE_IDENT_STRING=$HOSTNAME</code> as this most likely does not inlcude spaces.</li>
+</ol>
+</li>
+	<li>HBase uses the ./conf/<code><strong>hbase-default.xml</strong></code> file for configuration. Some properties do not resolve to existing directories because the JVM runs on Windows. This is the major issue to keep in mind when working with Cygwin: within the shell all paths are *nix-alike, hence relative to the root <code>/</code>. However, every parameter that is to be consumed within the windows processes themself, need to be Windows settings, hence <code>C:\</code>-alike. Change following propeties in the configuration file, adjusting paths where necessary to conform with your own installation:
+<ol>
+	<li><code>hbase.rootdir</code> must read e.g. <code>file:///C:/cygwin/root/tmp/hbase/data</code></li>
+	<li><code>hbase.tmp.dir</code> must read <code>C:/cygwin/root/tmp/hbase/tmp</code></li>
+	<li><code>hbase.zookeeper.quorum</code> must read <code>127.0.0.1</code> because for some reason <code>localhost</code> doesn't seem to resolve properly on Cygwin.</li>
+</ol>
+</li>
+	<li>Make sure the configured <code>hbase.rootdir</code> and <code>hbase.tmp.dir</code> <strong>directories exist</strong> and have the proper<strong> rights</strong> set up e.g. by issuing a <code>chmod 777</code> on them.</li>
+</ol>
+</section>
+</section>
+<section>
+<title>Testing</title>
+<p>
+This should conclude the installation and configuration of Apache HBase on Windows using Cygwin. So it's time <strong>to test it</strong>.
+<ol>
+	<li>Start a Cygwin<strong> terminal</strong>, if you haven't already.</li>
+	<li>Change directory to HBase <strong>installation</strong> using <code>CD /usr/local/hbase-<em>&lt;version&gt;</em></code>, preferably using auto-completion.</li>
+	<li><strong>Start HBase</strong> using the command <code>./bin/start-hbase.sh</code>
+<ol>
+	<li>When prompted to accept the SSH fingerprint, answer <code>yes</code>.</li>
+	<li>When prompted, provide your password. Maybe multiple times.</li>
+	<li>When the command completes, the HBase server should have started.</li>
+	<li>However, to be absolutely certain, check the logs in the <code>./logs</code> directory for any exceptions.</li>
+</ol>
+</li>
+	<li>Next we <strong>start the HBase shell</strong> using the command <code>./bin/hbase shell</code></li>
+	<li>We run some simple <strong>test commands</strong>
+<ol>
+	<li>Create a simple table using command <code>create 'test', 'data'</code></li>
+	<li>Verify the table exists using the command <code>list</code></li>
+	<li>Insert data into the table using e.g.
+<pre>put 'test', 'row1', 'data:1', 'value1'
+put 'test', 'row2', 'data:2', 'value2'
+put 'test', 'row3', 'data:3', 'value3'</pre>
+</li>
+	<li>List all rows in the table using the command <code>scan 'test'</code> that should list all the rows previously inserted. Notice how 3 new columns where added without changing the schema!</li>
+	<li>Finally we get rid of the table by issuing <code>disable 'test'</code> followed by <code>drop 'test'</code> and verified by <code>list</code> which should give an empty listing.</li>
+</ol>
+</li>
+	<li><strong>Leave the shell</strong> by <code>exit</code></li>
+	<li>To <strong>stop the HBase server</strong> issue the <code>./bin/stop-hbase.sh</code> command. And wait for it to complete!!! Killing the process might corrupt your data on disk.</li>
+	<li>In case of <strong>problems</strong>,
+<ol>
+	<li>verify the HBase logs in the <code>./logs</code> directory.</li>
+	<li>Try to fix the problem</li>
+	<li>Get help on the forums or IRC (<code>#hbase@freenode.net</code>). People are very active and keen to help out!</li>
+	<li>Stopr, restart and retest the server.</li>
+</ol>
+</li>
+</ol>
+</p>
+</section>
+
+<section name="Conclusion">
+<p>
+Now your <strong>HBase </strong>server is running, <strong>start coding</strong> and build that next killer app on this particular, but scalable datastore!
+</p>
+</section>
+</body>
+</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/xdoc/export_control.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/export_control.xml b/src/site/xdoc/export_control.xml
new file mode 100644
index 0000000..0fd5c4f
--- /dev/null
+++ b/src/site/xdoc/export_control.xml
@@ -0,0 +1,59 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title>
+      Export Control
+    </title>
+  </properties>
+  <body>
+  <section name="Export Control">
+<p>
+This distribution uses or includes cryptographic software. The country in
+which you currently reside may have restrictions on the import, possession,
+use, and/or re-export to another country, of encryption software. BEFORE
+using any encryption software, please check your country's laws, regulations
+and policies concerning the import, possession, or use, and re-export of
+encryption software, to see if this is permitted. See the
+<a href="http://www.wassenaar.org/">Wassenaar Arrangement</a> for more
+information.</p>
+<p>
+The U.S. Government Department of Commerce, Bureau of Industry and Security 
+(BIS), has classified this software as Export Commodity Control Number (ECCN) 
+5D002.C.1, which includes information security software using or performing 
+cryptographic functions with asymmetric algorithms. The form and manner of this
+Apache Software Foundation distribution makes it eligible for export under the 
+License Exception ENC Technology Software Unrestricted (TSU) exception (see the
+BIS Export Administration Regulations, Section 740.13) for both object code and
+source code.</p>
+<p>
+Apache HBase uses the built-in java cryptography libraries. See Oracle's
+information regarding
+<a href="http://www.oracle.com/us/products/export/export-regulations-345813.html">Java cryptographic export regulations</a>
+for more details.</p>
+  </section>
+  </body>
+</document>

[16/27] hbase git commit: HBASE-18485 Performance issue: ClientAsyncPrefetchScanner is slower than ClientSimpleScanner

Posted by bu...@apache.org.
HBASE-18485 Performance issue: ClientAsyncPrefetchScanner is slower than ClientSimpleScanner


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

Branch: refs/heads/HBASE-18467
Commit: 5915d73a70eb69adc639062f372559c9fc5130be
Parents: 2a71745
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Aug 4 23:10:18 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Aug 7 10:35:19 2017 +0800

----------------------------------------------------------------------
 .../client/ClientAsyncPrefetchScanner.java      | 181 ++++++-------------
 .../hadoop/hbase/client/ClientScanner.java      |   6 +-
 .../client/TestScannersFromClientSide.java      |   1 +
 3 files changed, 64 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5915d73a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
index 34c5620..e8da18f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
@@ -18,15 +18,20 @@
 package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
 import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
 
 import org.apache.hadoop.conf.Configuration;
@@ -48,25 +53,18 @@ import org.apache.hadoop.hbase.util.Threads;
 @InterfaceAudience.Private
 public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
 
-  private static final int ESTIMATED_SINGLE_RESULT_SIZE = 1024;
-  private static final int DEFAULT_QUEUE_CAPACITY = 1024;
-
-  private int cacheCapacity;
+  private long maxCacheSize;
   private AtomicLong cacheSizeInBytes;
   // exception queue (from prefetch to main scan execution)
   private Queue<Exception> exceptionsQueue;
-  // prefetch runnable object to be executed asynchronously
-  private PrefetchRunnable prefetchRunnable;
-  // Boolean flag to ensure only a single prefetch is running (per scan)
-  // We use atomic boolean to allow multiple concurrent threads to
-  // consume records from the same cache, but still have a single prefetcher thread.
-  // For a single consumer thread this can be replace with a native boolean.
-  private AtomicBoolean prefetchRunning;
-  // an attribute for synchronizing close between scanner and prefetch threads
-  private AtomicLong closingThreadId;
+  // prefetch thread to be executed asynchronously
+  private Thread prefetcher;
   // used for testing
   private Consumer<Boolean> prefetchListener;
-  private static final int NO_THREAD = -1;
+
+  private final Lock lock = new ReentrantLock();
+  private final Condition notEmpty = lock.newCondition();
+  private final Condition notFull = lock.newCondition();
 
   public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
       ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
@@ -84,82 +82,56 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
   @Override
   protected void initCache() {
     // concurrent cache
-    cacheCapacity = calcCacheCapacity();
+    maxCacheSize = resultSize2CacheSize(maxScannerResultSize);
     cache = new LinkedBlockingQueue<>();
     cacheSizeInBytes = new AtomicLong(0);
     exceptionsQueue = new ConcurrentLinkedQueue<>();
-    prefetchRunnable = new PrefetchRunnable();
-    prefetchRunning = new AtomicBoolean(false);
-    closingThreadId = new AtomicLong(NO_THREAD);
+    prefetcher = new Thread(new PrefetchRunnable());
+    Threads.setDaemonThreadRunning(prefetcher, tableName + ".asyncPrefetcher");
+  }
+
+  private long resultSize2CacheSize(long maxResultSize) {
+    // * 2 if possible
+    return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
   }
 
   @Override
   public Result next() throws IOException {
-
     try {
-      boolean hasExecutedPrefetch = false;
-      do {
+      lock.lock();
+      while (cache.isEmpty()) {
         handleException();
-
-        // If the scanner is closed and there's nothing left in the cache, next is a no-op.
-        if (getCacheCount() == 0 && this.closed) {
+        if (this.closed) {
           return null;
         }
-
-        if (prefetchCondition()) {
-          // run prefetch in the background only if no prefetch is already running
-          if (!isPrefetchRunning()) {
-            if (prefetchRunning.compareAndSet(false, true)) {
-              getPool().execute(prefetchRunnable);
-              hasExecutedPrefetch = true;
-            }
-          }
-        }
-
-        while (isPrefetchRunning()) {
-          // prefetch running or still pending
-          if (getCacheCount() > 0) {
-            return pollCache();
-          } else {
-            // (busy) wait for a record - sleep
-            Threads.sleep(1);
-          }
+        try {
+          notEmpty.await();
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException("Interrupted when wait to load cache");
         }
+      }
 
-        if (getCacheCount() > 0) {
-          return pollCache();
-        }
-      } while (!hasExecutedPrefetch);
-
-      // if we exhausted this scanner before calling close, write out the scan metrics
-      writeScanMetrics();
-      return null;
+      Result result = pollCache();
+      if (prefetchCondition()) {
+        notFull.signalAll();
+      }
+      return result;
     } finally {
+      lock.unlock();
       handleException();
     }
   }
 
   @Override
   public void close() {
-    if (!scanMetricsPublished) writeScanMetrics();
-    closed = true;
-    if (!isPrefetchRunning()) {
-      if(closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
-        super.close();
-      }
-    } // else do nothing since the async prefetch still needs this resources
-  }
-
-  @Override
-  public int getCacheCount() {
-    if(cache != null) {
-      int size = cache.size();
-      if(size > cacheCapacity) {
-        cacheCapacity = size;
-      }
-      return size;
-    } else {
-      return 0;
+    try {
+      lock.lock();
+      super.close();
+      closed = true;
+      notFull.signalAll();
+      notEmpty.signalAll();
+    } finally {
+      lock.unlock();
     }
   }
 
@@ -182,44 +154,8 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
     }
   }
 
-  private boolean isPrefetchRunning() {
-    return prefetchRunning.get();
-  }
-
-  // double buffer - double cache size
-  private int calcCacheCapacity() {
-    int capacity = Integer.MAX_VALUE;
-    if(caching > 0 && caching < (Integer.MAX_VALUE /2)) {
-      capacity = caching * 2 + 1;
-    }
-    if(capacity == Integer.MAX_VALUE){
-      if(maxScannerResultSize != Integer.MAX_VALUE) {
-        capacity = (int) (maxScannerResultSize / ESTIMATED_SINGLE_RESULT_SIZE);
-      }
-      else {
-        capacity = DEFAULT_QUEUE_CAPACITY;
-      }
-    }
-    return Math.max(capacity, 1);
-  }
-
   private boolean prefetchCondition() {
-    return
-        (getCacheCount() < getCountThreshold()) &&
-        (maxScannerResultSize == Long.MAX_VALUE ||
-         getCacheSizeInBytes() < getSizeThreshold()) ;
-  }
-
-  private int getCountThreshold() {
-    return Math.max(cacheCapacity / 2, 1);
-  }
-
-  private long getSizeThreshold() {
-    return Math.max(maxScannerResultSize / 2, 1);
-  }
-
-  private long getCacheSizeInBytes() {
-    return cacheSizeInBytes.get();
+    return cacheSizeInBytes.get() < maxCacheSize / 2;
   }
 
   private Result pollCache() {
@@ -233,21 +169,22 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
 
     @Override
     public void run() {
-      boolean succeed = false;
-      try {
-        loadCache();
-        succeed = true;
-      } catch (Exception e) {
-        exceptionsQueue.add(e);
-      } finally {
-        if (prefetchListener != null) {
-          prefetchListener.accept(succeed);
-        }
-        prefetchRunning.set(false);
-        if(closed) {
-          if (closingThreadId.compareAndSet(NO_THREAD, Thread.currentThread().getId())) {
-            // close was waiting for the prefetch to end
-            close();
+      while (!closed) {
+        boolean succeed = false;
+        try {
+          lock.lock();
+          while (!prefetchCondition()) {
+            notFull.await();
+          }
+          loadCache();
+          succeed = true;
+        } catch (Exception e) {
+          exceptionsQueue.add(e);
+        } finally {
+          notEmpty.signalAll();
+          lock.unlock();
+          if (prefetchListener != null) {
+            prefetchListener.accept(succeed);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5915d73a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index d3b19e4..2522434 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -72,7 +72,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
   protected Result lastResult = null;
   protected final long maxScannerResultSize;
   private final ClusterConnection connection;
-  private final TableName tableName;
+  protected final TableName tableName;
   protected final int scannerTimeout;
   protected boolean scanMetricsPublished = false;
   protected RpcRetryingCaller<Result[]> caller;
@@ -412,6 +412,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
     // This is possible if we just stopped at the boundary of a region in the previous call.
     if (callable == null) {
       if (!moveToNextRegion()) {
+        closed = true;
         return;
       }
     }
@@ -478,7 +479,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
         assert newLimit >= 0;
         scan.setLimit(newLimit);
       }
-      if (scanExhausted(values)) {
+      if (scan.getLimit() == 0 || scanExhausted(values)) {
         closeScanner();
         closed = true;
         break;
@@ -532,6 +533,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
       // we are done with the current region
       if (regionExhausted) {
         if (!moveToNextRegion()) {
+          closed = true;
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5915d73a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index ef00b24..43be573 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -754,6 +754,7 @@ public class TestScannersFromClientSide {
       result = Result.create(kvListScan);
       verifyResult(result, kvListExp, toLog, "Testing async scan");
     }
+
     TEST_UTIL.deleteTable(table);
   }
 


[18/27] hbase git commit: HBASE-18502 Change MasterObserver to use TableDescriptor and ColumnFamilyDescriptor

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
index bd9d5dc..5da0d94 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotClientRetries.java
@@ -26,8 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@@ -80,7 +80,7 @@ public class TestSnapshotClientRetries {
 
     @Override
     public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       if (snapshotCount != null) {
         snapshotCount.incrementAndGet();
@@ -89,7 +89,7 @@ public class TestSnapshotClientRetries {
 
     @Override
     public void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor)
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
         throws IOException {
       if (cloneCount != null) {
         cloneCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd76eb39/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index f6e328e..c18d6d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -55,9 +55,9 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -598,7 +598,7 @@ public class BaseTestHBaseFsck {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HTableDescriptor desc,
+        final TableDescriptor desc,
         final HRegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()


[05/27] hbase git commit: HBASE-18516 Removed dead code in ServerManager resulted mostly from AMv2 refactoring

Posted by bu...@apache.org.
HBASE-18516 Removed dead code in ServerManager resulted mostly from AMv2 refactoring

* Call to methods sendRegionOpen(), isServerReachable(), removeRequeuedDeadServers(), getRequeuedDeadServers() got removed in HBASE-14614
* Call to method ServerManager.sendFavoredNodes() got removed in HBASE-17198


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

Branch: refs/heads/HBASE-18467
Commit: 62deb8172e5a2d3da761bb5b55dfd02f5662f47e
Parents: 2bde7a1
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Thu Aug 3 13:17:56 2017 -0700
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Fri Aug 4 13:47:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/master/ServerManager.java      | 223 +------------------
 1 file changed, 10 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/62deb817/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index c636a67..c9c792a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
 import java.io.IOException;
-import java.net.ConnectException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -53,31 +52,18 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.RetryCounterFactory;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -157,7 +143,6 @@ public class ServerManager {
   private final long maxSkew;
   private final long warningSkew;
 
-  private final RetryCounterFactory pingRetryCounterFactory;
   private final RpcControllerFactory rpcControllerFactory;
 
   /**
@@ -203,21 +188,16 @@ public class ServerManager {
    * @param master
    * @throws ZooKeeperConnectionException
    */
-  public ServerManager(final MasterServices master) throws IOException {
+  public ServerManager(final MasterServices master) {
     this(master, true);
   }
 
-  ServerManager(final MasterServices master, final boolean connect) throws IOException {
+  ServerManager(final MasterServices master, final boolean connect) {
     this.master = master;
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
     this.connection = connect ? master.getClusterConnection() : null;
-    int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
-      "hbase.master.maximum.ping.server.attempts", 10));
-    int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
-      "hbase.master.ping.server.retry.sleep.interval", 100));
-    this.pingRetryCounterFactory = new RetryCounterFactory(pingMaxAttempts, pingSleepInterval);
     this.rpcControllerFactory = this.connection == null
         ? null
         : connection.getRpcControllerFactory();
@@ -251,7 +231,7 @@ public class ServerManager {
     // Test for case where we get a region startup message from a regionserver
     // that has been quickly restarted but whose znode expiration handler has
     // not yet run, or from a server whose fail we are currently processing.
-    // Test its host+port combo is present in serverAddresstoServerInfo.  If it
+    // Test its host+port combo is present in serverAddressToServerInfo.  If it
     // is, reject the server and trigger its expiration. The next time it comes
     // in, it should have been removed from serverAddressToServerInfo and queued
     // for processing by ProcessServerShutdown.
@@ -443,7 +423,7 @@ public class ServerManager {
   /**
    * Adds the onlineServers list. onlineServers should be locked.
    * @param serverName The remote servers name.
-   * @param s
+   * @param sl
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
@@ -722,117 +702,6 @@ public class ServerManager {
 
   // RPC methods to region servers
 
-  /**
-   * Sends an OPEN RPC to the specified server to open the specified region.
-   * <p>
-   * Open should not fail but can if server just crashed.
-   * <p>
-   * @param server server to open a region
-   * @param region region to open
-   * @param favoredNodes
-   */
-  public RegionOpeningState sendRegionOpen(final ServerName server,
-      HRegionInfo region, List<ServerName> favoredNodes)
-  throws IOException {
-    AdminService.BlockingInterface admin = getRsAdmin(server);
-    if (admin == null) {
-      throw new IOException("Attempting to send OPEN RPC to server " + server.toString() +
-        " failed because no RPC connection found to this server");
-    }
-    OpenRegionRequest request =
-        RequestConverter.buildOpenRegionRequest(server, region, favoredNodes, false);
-    try {
-      OpenRegionResponse response = admin.openRegion(null, request);
-      return ResponseConverter.getRegionOpeningState(response);
-    } catch (ServiceException se) {
-      checkForRSznode(server, se);
-      throw ProtobufUtil.getRemoteException(se);
-    }
-  }
-
-  /**
-   * Check for an odd state, where we think an RS is up but it is not. Do it on OPEN.
-   * This is only case where the check makes sense.
-   *
-   * <p>We are checking for instance of HBASE-9593 where a RS registered but died before it put
-   * up its znode in zk. In this case, the RS made it into the list of online servers but it
-   * is not actually UP. We do the check here where there is an evident problem rather
-   * than do some crazy footwork where we'd have master check zk after a RS had reported
-   * for duty with provisional state followed by a confirmed state; that'd be a mess.
-   * Real fix is HBASE-17733.
-   */
-  private void checkForRSznode(final ServerName serverName, final ServiceException se) {
-    if (se.getCause() == null) return;
-    Throwable t = se.getCause();
-    if (t instanceof ConnectException) {
-      // If this, proceed to do cleanup.
-    } else {
-      // Look for FailedServerException
-      if (!(t instanceof IOException)) return;
-      if (t.getCause() == null) return;
-      if (!(t.getCause() instanceof FailedServerException)) return;
-      // Ok, found FailedServerException -- continue.
-    }
-    if (!isServerOnline(serverName)) return;
-    // We think this server is online. Check it has a znode up. Currently, a RS
-    // registers an ephereral znode in zk. If not present, something is up. Maybe
-    // HBASE-9593 where RS crashed AFTER reportForDuty but BEFORE it put up an ephemeral
-    // znode.
-    List<String> servers = null;
-    try {
-      servers = getRegionServersInZK(this.master.getZooKeeper());
-    } catch (KeeperException ke) {
-      LOG.warn("Failed to list regionservers", ke);
-      // ZK is malfunctioning, don't hang here
-    }
-    boolean found = false;
-    if (servers != null) {
-      for (String serverNameAsStr: servers) {
-        ServerName sn = ServerName.valueOf(serverNameAsStr);
-        if (sn.equals(serverName)) {
-          // Found a server up in zk.
-          found = true;
-          break;
-        }
-      }
-    }
-    if (!found) {
-      LOG.warn("Online server " + serverName.toString() + " has no corresponding " +
-        "ephemeral znode (Did it die before registering in zk?); " +
-          "calling expire to clean it up!");
-      expireServer(serverName);
-    }
-  }
-
-  /**
-   * Sends an OPEN RPC to the specified server to open the specified region.
-   * <p>
-   * Open should not fail but can if server just crashed.
-   * <p>
-   * @param server server to open a region
-   * @param regionOpenInfos info of a list of regions to open
-   * @return a list of region opening states
-   */
-  public List<RegionOpeningState> sendRegionOpen(ServerName server,
-      List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos)
-  throws IOException {
-    AdminService.BlockingInterface admin = getRsAdmin(server);
-    if (admin == null) {
-      throw new IOException("Attempting to send OPEN RPC to server " + server.toString() +
-        " failed because no RPC connection found to this server");
-    }
-
-    OpenRegionRequest request =
-        RequestConverter.buildOpenRegionRequest(server, regionOpenInfos, false);
-    try {
-      OpenRegionResponse response = admin.openRegion(null, request);
-      return ResponseConverter.getRegionOpeningStateList(response);
-    } catch (ServiceException se) {
-      checkForRSznode(server, se);
-      throw ProtobufUtil.getRemoteException(se);
-    }
-  }
-
   private HBaseRpcController newRpcController() {
     return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
   }
@@ -892,41 +761,11 @@ public class ServerManager {
   }
 
   /**
-   * Check if a region server is reachable and has the expected start code
+   * @param sn
+   * @return Admin interface for the remote regionserver named <code>sn</code>
+   * @throws IOException
+   * @throws RetriesExhaustedException wrapping a ConnectException if failed
    */
-  public boolean isServerReachable(ServerName server) {
-    if (server == null) throw new NullPointerException("Passed server is null");
-
-
-    RetryCounter retryCounter = pingRetryCounterFactory.create();
-    while (retryCounter.shouldRetry()) {
-      try {
-        HBaseRpcController controller = newRpcController();
-        AdminService.BlockingInterface admin = getRsAdmin(server);
-        if (admin != null) {
-          ServerInfo info = ProtobufUtil.getServerInfo(controller, admin);
-          return info != null && info.hasServerName()
-            && server.getStartcode() == info.getServerName().getStartCode();
-        }
-      } catch (IOException ioe) {
-        LOG.debug("Couldn't reach " + server + ", try=" + retryCounter.getAttemptTimes()
-          + " of " + retryCounter.getMaxAttempts(), ioe);
-        try {
-          retryCounter.sleepUntilNextRetry();
-        } catch(InterruptedException ie) {
-          Thread.currentThread().interrupt();
-        }
-      }
-    }
-    return false;
-  }
-
-    /**
-    * @param sn
-    * @return Admin interface for the remote regionserver named <code>sn</code>
-    * @throws IOException
-    * @throws RetriesExhaustedException wrapping a ConnectException if failed
-    */
   public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
   throws IOException {
     AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
@@ -1088,23 +927,6 @@ public class ServerManager {
     return new HashSet<>(this.queuedDeadServers);
   }
 
-  /**
-   * During startup, if we figure it is not a failover, i.e. there is
-   * no more WAL files to split, we won't try to recover these dead servers.
-   * So we just remove them from the queue. Use caution in calling this.
-   */
-  void removeRequeuedDeadServers() {
-    requeuedDeadServers.clear();
-  }
-
-  /**
-   * @return A copy of the internal map of requeuedDeadServers servers and their corresponding
-   *         splitlog need flag.
-   */
-  Map<ServerName, Boolean> getRequeuedDeadServers() {
-    return Collections.unmodifiableMap(this.requeuedDeadServers);
-  }
-
   public boolean isServerOnline(ServerName serverName) {
     return serverName != null && onlineServers.containsKey(serverName);
   }
@@ -1157,11 +979,7 @@ public class ServerManager {
 
     // Loop through the draining server list and remove them from the server list
     final List<ServerName> drainingServersCopy = getDrainingServersList();
-    if (!drainingServersCopy.isEmpty()) {
-      for (final ServerName server: drainingServersCopy) {
-        destServers.remove(server);
-      }
-    }
+    destServers.removeAll(drainingServersCopy);
 
     // Remove the deadNotExpired servers from the server list.
     removeDeadNotExpiredServers(destServers);
@@ -1218,25 +1036,4 @@ public class ServerManager {
       removeRegion(hri);
     }
   }
-
-  public void sendFavoredNodes(final ServerName server,
-      Map<HRegionInfo, List<ServerName>> favoredNodes) throws IOException {
-    AdminService.BlockingInterface admin = getRsAdmin(server);
-    if (admin == null) {
-      LOG.warn("Attempting to send favored nodes update rpc to server " + server.toString()
-          + " failed because no RPC connection found to this server");
-    } else {
-      List<Pair<HRegionInfo, List<ServerName>>> regionUpdateInfos = new ArrayList<>();
-      for (Entry<HRegionInfo, List<ServerName>> entry : favoredNodes.entrySet()) {
-        regionUpdateInfos.add(new Pair<>(entry.getKey(), entry.getValue()));
-      }
-      UpdateFavoredNodesRequest request =
-        RequestConverter.buildUpdateFavoredNodesRequest(regionUpdateInfos);
-      try {
-        admin.updateFavoredNodes(null, request);
-      } catch (ServiceException se) {
-        throw ProtobufUtil.getRemoteException(se);
-      }
-    }
-  }
 }


[09/27] hbase git commit: HBASE-15042 Moved site materials to standard Maven place

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/metrics.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/metrics.xml b/src/main/site/xdoc/metrics.xml
deleted file mode 100644
index f3ab7d7..0000000
--- a/src/main/site/xdoc/metrics.xml
+++ /dev/null
@@ -1,150 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title> 
-      Apache HBase (TM) Metrics
-    </title>
-  </properties>
-
-  <body>
-    <section name="Introduction">
-      <p>
-      Apache HBase (TM) emits Hadoop <a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html">metrics</a>.
-      </p>
-      </section>
-      <section name="Setup">
-      <p>First read up on Hadoop <a href="http://hadoop.apache.org/core/docs/current/api/org/apache/hadoop/metrics/package-summary.html">metrics</a>.
-      If you are using ganglia, the <a href="http://wiki.apache.org/hadoop/GangliaMetrics">GangliaMetrics</a>
-      wiki page is useful read.</p>
-      <p>To have HBase emit metrics, edit <code>$HBASE_HOME/conf/hadoop-metrics.properties</code>
-      and enable metric 'contexts' per plugin.  As of this writing, hadoop supports
-      <strong>file</strong> and <strong>ganglia</strong> plugins.
-      Yes, the hbase metrics files is named hadoop-metrics rather than
-      <em>hbase-metrics</em> because currently at least the hadoop metrics system has the
-      properties filename hardcoded. Per metrics <em>context</em>,
-      comment out the NullContext and enable one or more plugins instead.
-      </p>
-      <p>
-      If you enable the <em>hbase</em> context, on regionservers you'll see total requests since last
-      metric emission, count of regions and storefiles as well as a count of memstore size.
-      On the master, you'll see a count of the cluster's requests.
-      </p>
-      <p>
-      Enabling the <em>rpc</em> context is good if you are interested in seeing
-      metrics on each hbase rpc method invocation (counts and time taken).
-      </p>
-      <p>
-      The <em>jvm</em> context is
-      useful for long-term stats on running hbase jvms -- memory used, thread counts, etc.
-      As of this writing, if more than one jvm is running emitting metrics, at least
-      in ganglia, the stats are aggregated rather than reported per instance.
-      </p>
-    </section>
-
-    <section name="Using with JMX">
-      <p>
-      In addition to the standard output contexts supported by the Hadoop 
-      metrics package, you can also export HBase metrics via Java Management 
-      Extensions (JMX).  This will allow viewing HBase stats in JConsole or 
-      any other JMX client.
-      </p>
-      <section name="Enable HBase stats collection">
-      <p>
-      To enable JMX support in HBase, first edit 
-      <code>$HBASE_HOME/conf/hadoop-metrics.properties</code> to support 
-      metrics refreshing. (If you've running 0.94.1 and above, or have already configured 
-      <code>hadoop-metrics.properties</code> for another output context,
-      you can skip this step).
-      </p>
-      <source>
-# Configuration of the "hbase" context for null
-hbase.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-hbase.period=60
-
-# Configuration of the "jvm" context for null
-jvm.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-jvm.period=60
-
-# Configuration of the "rpc" context for null
-rpc.class=org.apache.hadoop.metrics.spi.NullContextWithUpdateThread
-rpc.period=60
-      </source>
-      </section>
-      <section name="Setup JMX remote access">
-      <p>
-      For remote access, you will need to configure JMX remote passwords 
-      and access profiles.  Create the files:
-      </p>
-      <dl>
-        <dt><code>$HBASE_HOME/conf/jmxremote.passwd</code> (set permissions 
-        to 600)</dt>
-        <dd>
-        <source>
-monitorRole monitorpass
-controlRole controlpass
-        </source>
-        </dd>
-        
-        <dt><code>$HBASE_HOME/conf/jmxremote.access</code></dt>
-        <dd>
-        <source>
-monitorRole readonly
-controlRole readwrite
-        </source>
-        </dd>
-      </dl>
-      </section>
-      <section name="Configure JMX in HBase startup">
-      <p>
-      Finally, edit the <code>$HBASE_HOME/conf/hbase-env.sh</code>
-      script to add JMX support: 
-      </p>
-      <dl>
-        <dt><code>$HBASE_HOME/conf/hbase-env.sh</code></dt>
-        <dd>
-        <p>Add the lines:</p>
-        <source>
-HBASE_JMX_OPTS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false"
-HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.password.file=$HBASE_HOME/conf/jmxremote.passwd"
-HBASE_JMX_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.access.file=$HBASE_HOME/conf/jmxremote.access"
-
-export HBASE_MASTER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10101"
-export HBASE_REGIONSERVER_OPTS="$HBASE_JMX_OPTS -Dcom.sun.management.jmxremote.port=10102"
-        </source>
-        </dd>
-      </dl>
-      <p>
-      After restarting the processes you want to monitor, you should now be 
-      able to run JConsole (included with the JDK since JDK 5.0) to view 
-      the statistics via JMX.  HBase MBeans are exported under the 
-      <strong><code>hadoop</code></strong> domain in JMX.
-      </p>
-      </section>
-      <section name="Understanding HBase Metrics">
-      <p>
-      For more information on understanding HBase metrics, see the <a href="book.html#hbase_metrics">metrics section</a> in the Apache HBase Reference Guide. 
-      </p>
-      </section>
-    </section>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/old_news.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/old_news.xml b/src/main/site/xdoc/old_news.xml
deleted file mode 100644
index 94e1882..0000000
--- a/src/main/site/xdoc/old_news.xml
+++ /dev/null
@@ -1,92 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>
-      Old Apache HBase (TM) News
-    </title>
-  </properties>
-  <body>
-  <section name="Old News">
-         <p>February 10th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/163139322/">HBase Meetup @ Continuuity</a> in Palo Alto</p>
-         <p>January 30th, 2014 <a href="http://www.meetup.com/hbaseusergroup/events/158491762/">HBase Meetup @ Apple</a> in Cupertino</p>
-         <p>January 30th, 2014 <a href="http://www.meetup.com/Los-Angeles-HBase-User-group/events/160560282/">Los Angeles HBase User Group</a> in El Segundo</p>
-         <p>October 24th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/140759692/">HBase User and <a href="http://www.meetup.com/hackathon/events/144366512/">Developer</a> Meetup at HortonWorks</a>.in Palo Alto</p>
-         <p>September 26, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/135862292/">HBase Meetup at Arista Networks</a>.in San Francisco</p>
-         <p>August 20th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/120534362/">HBase Meetup at Flurry</a>.in San Francisco</p>
-         <p>July 16th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/119929152/">HBase Meetup at Twitter</a>.in San Francisco</p>
-         <p>June 25th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/119154442/">Hadoop Summit Meetup</a>.at San Jose Convention Center</p>
-         <p>June 14th, 2013 <a href="http://kijicon.eventbrite.com/">KijiCon: Building Big Data Apps</a> in San Francisco.</p>
-         <p>June 13th, 2013 <a href="http://www.hbasecon.com/">HBaseCon2013</a> in San Francisco.  Submit an Abstract!</p>
-         <p>June 12th, 2013 <a href="http://www.meetup.com/hackathon/events/123403802/">HBaseConHackAthon</a> at the Cloudera office in San Francisco.</p>
-         <p>April 11th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/103587852/">HBase Meetup at AdRoll</a> in San Francisco</p>
-         <p>February 28th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/96584102/">HBase Meetup at Intel Mission Campus</a></p>
-         <p>February 19th, 2013 <a href="http://www.meetup.com/hackathon/events/103633042/">Developers PowWow</a> at HortonWorks' new digs</p>
-         <p>January 23rd, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/91381312/">HBase Meetup at WibiData World HQ!</a></p>
-            <p>December 4th, 2012 <a href="http://www.meetup.com/hackathon/events/90536432/">0.96 Bug Squashing and Testing Hackathon</a> at Cloudera, SF.</p>
-            <p>October 29th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/82791572/">HBase User Group Meetup</a> at Wize Commerce in San Mateo.</p>
-            <p>October 25th, 2012 <a href="http://www.meetup.com/HBase-NYC/events/81728932/">Strata/Hadoop World HBase Meetup.</a> in NYC</p>
-            <p>September 11th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/80621872/">Contributor's Pow-Wow at HortonWorks HQ.</a></p>
-            <p>August 8th, 2012 <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Apache HBase 0.94.1 is available for download</a></p>
-            <p>June 15th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/59829652/">Birds-of-a-feather</a> in San Jose, day after <a href="http://hadoopsummit.org">Hadoop Summit</a></p>
-            <p>May 23rd, 2012 <a href="http://www.meetup.com/hackathon/events/58953522/">HackConAthon</a> in Palo Alto</p>
-            <p>May 22nd, 2012 <a href="http://www.hbasecon.com">HBaseCon2012</a> in San Francisco</p>
-            <p>March 27th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/56021562/">Meetup @ StumbleUpon</a> in San Francisco</p>
-
-            <p>January 19th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/46702842/">Meetup @ EBay</a></p>
-            <p>January 23rd, 2012 Apache HBase 0.92.0 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
-            <p>December 23rd, 2011 Apache HBase 0.90.5 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
-            <p>November 29th, 2011 <a href="http://www.meetup.com/hackathon/events/41025972/">Developer Pow-Wow in SF</a> at Salesforce HQ</p>
-            <p>November 7th, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/35682812/">HBase Meetup in NYC (6PM)</a> at the AppNexus office</p>
-            <p>August 22nd, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/28518471/">HBase Hackathon (11AM) and Meetup (6PM)</a> at FB in PA</p>
-            <p>June 30th, 2011 <a href="http://www.meetup.com/hbaseusergroup/events/20572251/">HBase Contributor Day</a>, the day after the <a href="http://developer.yahoo.com/events/hadoopsummit2011/">Hadoop Summit</a> hosted by Y!</p>
-            <p>June 8th, 2011 <a href="http://berlinbuzzwords.de/wiki/hbase-workshop-and-hackathon">HBase Hackathon</a> in Berlin to coincide with <a href="http://berlinbuzzwords.de/">Berlin Buzzwords</a></p>
-            <p>May 19th, 2011 Apache HBase 0.90.3 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
-            <p>April 12th, 2011 Apache HBase 0.90.2 released. <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Download it!</a></p>
-            <p>March 21st, <a href="http://www.meetup.com/hackathon/events/16770852/">HBase 0.92 Hackathon at StumbleUpon, SF</a></p>
-            <p>February 22nd, <a href="http://www.meetup.com/hbaseusergroup/events/16492913/">HUG12: February HBase User Group at StumbleUpon SF</a></p>
-            <p>December 13th, <a href="http://www.meetup.com/hackathon/calendar/15597555/">HBase Hackathon: Coprocessor Edition</a></p>
-      <p>November 19th, <a href="http://huguk.org/">Hadoop HUG in London</a> is all about Apache HBase</p>
-      <p>November 15-19th, <a href="http://www.devoxx.com/display/Devoxx2K10/Home">Devoxx</a> features HBase Training and multiple HBase presentations</p>
-      <p>October 12th, HBase-related presentations by core contributors and users at <a href="http://www.cloudera.com/company/press-center/hadoop-world-nyc/">Hadoop World 2010</a></p>
-      <p>October 11th, <a href="http://www.meetup.com/hbaseusergroup/calendar/14606174/">HUG-NYC: HBase User Group NYC Edition</a> (Night before Hadoop World)</p>
-      <p>June 30th, <a href="http://www.meetup.com/hbaseusergroup/calendar/13562846/">Apache HBase Contributor Workshop</a> (Day after Hadoop Summit)</p>
-      <p>May 10th, 2010: Apache HBase graduates from Hadoop sub-project to Apache Top Level Project </p>
-      <p>Signup for <a href="http://www.meetup.com/hbaseusergroup/calendar/12689490/">HBase User Group Meeting, HUG10</a> hosted by Trend Micro, April 19th, 2010</p>
-
-      <p><a href="http://www.meetup.com/hbaseusergroup/calendar/12689351/">HBase User Group Meeting, HUG9</a> hosted by Mozilla, March 10th, 2010</p>
-      <p>Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/12241393/">HBase User Group Meeting, HUG8</a>, January 27th, 2010 at StumbleUpon in SF</p>
-      <p>September 8th, 2010: Apache HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous Apache HBase release.  Get it off the <a href="http://www.apache.org/dyn/closer.cgi/hbase/">Releases</a> page.</p>
-      <p><a href="http://dev.us.apachecon.com/c/acus2009/">ApacheCon</a> in Oakland: November 2-6th, 2009:
-      The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.</p>
-      <p>HBase at Hadoop World in NYC: October 2nd, 2009: A few of us will be talking on Practical HBase out east at <a href="http://www.cloudera.com/hadoop-world-nyc">Hadoop World: NYC</a>.</p>
-      <p>HUG7 and HBase Hackathon: August 7th-9th, 2009 at StumbleUpon in SF: Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/10950511/">HBase User Group Meeting, HUG7</a> or for the <a href="http://www.meetup.com/hackathon/calendar/10951718/">Hackathon</a> or for both (all are welcome!).</p>
-      <p>June, 2009 -- HBase at HadoopSummit2009 and at NOSQL: See the <a href="http://wiki.apache.org/hadoop/HBase/HBasePresentations">presentations</a></p>
-      <p>March 3rd, 2009 -- HUG6: <a href="http://www.meetup.com/hbaseusergroup/calendar/9764004/">HBase User Group 6</a></p>
-      <p>January 30th, 2009 -- LA Hbackathon:<a href="http://www.meetup.com/hbasela/calendar/9450876/">HBase January Hackathon Los Angeles</a> at <a href="http://streamy.com" >Streamy</a> in Manhattan Beach</p>
-  </section>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/poweredbyhbase.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/poweredbyhbase.xml b/src/main/site/xdoc/poweredbyhbase.xml
deleted file mode 100644
index ff1ba59..0000000
--- a/src/main/site/xdoc/poweredbyhbase.xml
+++ /dev/null
@@ -1,398 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Powered By Apache HBase&#153;</title>
-  </properties>
-
-<body>
-<section name="Powered By Apache HBase&#153;">
-  <p>This page lists some institutions and projects which are using HBase. To
-    have your organization added, file a documentation JIRA or email
-    <a href="mailto:dev@hbase.apache.org">hbase-dev</a> with the relevant
-    information. If you notice out-of-date information, use the same avenues to
-    report it.
-  </p>
-  <p><b>These items are user-submitted and the HBase team assumes no responsibility for their accuracy.</b></p>
-  <dl>
-  <dt><a href="http://www.adobe.com">Adobe</a></dt>
-  <dd>We currently have about 30 nodes running HDFS, Hadoop and HBase  in clusters
-    ranging from 5 to 14 nodes on both production and development. We plan a
-    deployment on an 80 nodes cluster. We are using HBase in several areas from
-    social services to structured data and processing for internal use. We constantly
-    write data to HBase and run mapreduce jobs to process then store it back to
-    HBase or external systems. Our production cluster has been running since Oct 2008.</dd>
-
-  <dt><a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Project Astro</a></dt>
-  <dd>
-    Astro provides fast Spark SQL/DataFrame capabilities to HBase data,
-    featuring super-efficient access to multi-dimensional HBase rows through
-    native Spark execution in HBase coprocessor plus systematic and accurate
-    partition pruning and predicate pushdown from arbitrarily complex data
-    filtering logic. The batch load is optimized to run on the Spark execution
-    engine. Note that <a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Spark-SQL-on-HBase</a>
-    is the release site. Interested parties are free to make clones and claim
-    to be "latest(and active)", but they are not endorsed by the owner.
-  </dd>
-
-  <dt><a href="http://axibase.com/products/axibase-time-series-database/">Axibase
-    Time Series Database (ATSD)</a></dt>
-  <dd>ATSD runs on top of HBase to collect, analyze and visualize time series
-    data at scale. ATSD capabilities include optimized storage schema, built-in
-    rule engine, forecasting algorithms (Holt-Winters and ARIMA) and next-generation
-    graphics designed for high-frequency data. Primary use cases: IT infrastructure
-    monitoring, data consolidation, operational historian in OPC environments.</dd>
-
-  <dt><a href="http://www.benipaltechnologies.com">Benipal Technologies</a></dt>
-  <dd>We have a 35 node cluster used for HBase and Mapreduce with Lucene / SOLR
-    and katta integration to create and finetune our search databases. Currently,
-    our HBase installation has over 10 Billion rows with 100s of datapoints per row.
-    We compute over 10<sup>18</sup> calculations daily using MapReduce directly on HBase. We
-    heart HBase.</dd>
-
-  <dt><a href="https://github.com/ermanpattuk/BigSecret">BigSecret</a></dt>
-  <dd>BigSecret is a security framework that is designed to secure Key-Value data,
-    while preserving efficient processing capabilities. It achieves cell-level
-    security, using combinations of different cryptographic techniques, in an
-    efficient and secure manner. It provides a wrapper library around HBase.</dd>
-
-  <dt><a href="http://caree.rs">Caree.rs</a></dt>
-  <dd>Accelerated hiring platform for HiTech companies. We use HBase and Hadoop
-    for all aspects of our backend - job and company data storage, analytics
-    processing, machine learning algorithms for our hire recommendation engine.
-    Our live production site is directly served from HBase. We use cascading for
-    running offline data processing jobs.</dd>
-
-  <dt><a href="http://www.celer-tech.com/">Celer Technologies</a></dt>
-  <dd>Celer Technologies is a global financial software company that creates
-    modular-based systems that have the flexibility to meet tomorrow's business
-    environment, today.  The Celer framework uses Hadoop/HBase for storing all
-    financial data for trading, risk, clearing in a single data store. With our
-    flexible framework and all the data in Hadoop/HBase, clients can build new
-    features to quickly extract data based on their trading, risk and clearing
-    activities from one single location.</dd>
-
-  <dt><a href="http://www.explorys.net">Explorys</a></dt>
-  <dd>Explorys uses an HBase cluster containing over a billion anonymized clinical
-    records, to enable subscribers to search and analyze patient populations,
-    treatment protocols, and clinical outcomes.</dd>
-
-  <dt><a href="http://www.facebook.com/notes/facebook-engineering/the-underlying-technology-of-messages/454991608919">Facebook</a></dt>
-  <dd>Facebook uses HBase to power their Messages infrastructure.</dd>
-
-  <dt><a href="http://www.filmweb.pl">Filmweb</a></dt>
-  <dd>Filmweb is a film web portal with a large dataset of films, persons and
-    movie-related entities. We have just started a small cluster of 3 HBase nodes
-    to handle our web cache persistency layer. We plan to increase the cluster
-    size, and also to start migrating some of the data from our databases which
-    have some demanding scalability requirements.</dd>
-
-  <dt><a href="http://www.flurry.com">Flurry</a></dt>
-  <dd>Flurry provides mobile application analytics. We use HBase and Hadoop for
-    all of our analytics processing, and serve all of our live requests directly
-    out of HBase on our 50 node production cluster with tens of billions of rows
-    over several tables.</dd>
-
-  <dt><a href="http://gumgum.com">GumGum</a></dt>
-  <dd>GumGum is an In-Image Advertising Platform. We use HBase on an 15-node
-    Amazon EC2 High-CPU Extra Large (c1.xlarge) cluster for both real-time data
-    and analytics. Our production cluster has been running since June 2010.</dd>
-
-  <dt><a href="http://helprace.com/help-desk/">Helprace</a></dt>
-  <dd>Helprace is a customer service platform which uses Hadoop for analytics
-    and internal searching and filtering. Being on HBase we can share our HBase
-    and Hadoop cluster with other Hadoop processes - this particularly helps in
-    keeping community speeds up. We use Hadoop and HBase on small cluster with 4
-    cores and 32 GB RAM each.</dd>
-
-  <dt><a href="http://hubspot.com">HubSpot</a></dt>
-  <dd>HubSpot is an online marketing platform, providing analytics, email, and
-    segmentation of leads/contacts.  HBase is our primary datastore for our customers'
-    customer data, with multiple HBase clusters powering the majority of our
-    product.  We have nearly 200 regionservers across the various clusters, and
-    2 hadoop clusters also with nearly 200 tasktrackers.  We use c1.xlarge in EC2
-    for both, but are starting to move some of that to baremetal hardware.  We've
-    been running HBase for over 2 years.</dd>
-
-  <dt><a href="http://www.infolinks.com/">Infolinks</a></dt>
-  <dd>Infolinks is an In-Text ad provider. We use HBase to process advertisement
-    selection and user events for our In-Text ad network. The reports generated
-    from HBase are used as feedback for our production system to optimize ad
-    selection.</dd>
-
-  <dt><a href="http://www.kalooga.com">Kalooga</a></dt>
-  <dd>Kalooga is a discovery service for image galleries. We use Hadoop, HBase
-    and Pig on a 20-node cluster for our crawling, analysis and events
-    processing.</dd>
-
-  <dt><a href="http://www.leanxcale.com/">LeanXcale</a></dt>
-  <dd>LeanXcale provides an ultra-scalable transactional &amp; SQL database that
-  stores its data on HBase and it is able to scale to 1000s of nodes. It
-  also provides a standalone full ACID HBase with transactions across
-  arbitrary sets of rows and tables.</dd>
-
-
-  <dt><a href="http://www.mahalo.com">Mahalo</a></dt>
-  <dd>Mahalo, "...the world's first human-powered search engine". All the markup
-    that powers the wiki is stored in HBase. It's been in use for a few months now.
-    MediaWiki - the same software that power Wikipedia - has version/revision control.
-    Mahalo's in-house editors produce a lot of revisions per day, which was not
-    working well in a RDBMS. An hbase-based solution for this was built and tested,
-    and the data migrated out of MySQL and into HBase. Right now it's at something
-    like 6 million items in HBase. The upload tool runs every hour from a shell
-    script to back up that data, and on 6 nodes takes about 5-10 minutes to run -
-    and does not slow down production at all.</dd>
-
-  <dt><a href="http://www.meetup.com">Meetup</a></dt>
-  <dd>Meetup is on a mission to help the world’s people self-organize into local
-    groups.  We use Hadoop and HBase to power a site-wide, real-time activity
-    feed system for all of our members and groups.  Group activity is written
-    directly to HBase, and indexed per member, with the member's custom feed
-    served directly from HBase for incoming requests.  We're running HBase
-    0.20.0 on a 11 node cluster.</dd>
-
-  <dt><a href="http://www.mendeley.com">Mendeley</a></dt>
-  <dd>Mendeley is creating a platform for researchers to collaborate and share
-    their research online. HBase is helping us to create the world's largest
-    research paper collection and is being used to store all our raw imported data.
-    We use a lot of map reduce jobs to process these papers into pages displayed
-    on the site. We also use HBase with Pig to do analytics and produce the article
-    statistics shown on the web site. You can find out more about how we use HBase
-    in the <a href="http://www.slideshare.net/danharvey/hbase-at-mendeley">HBase
-    At Mendeley</a> slide presentation.</dd>
-
-  <dt><a href="http://www.ngdata.com">NGDATA</a></dt>
-  <dd>NGDATA delivers <a href="http://www.ngdata.com/site/products/lily.html">Lily</a>,
-    the consumer intelligence solution that delivers a unique combination of Big
-    Data management, machine learning technologies and consumer intelligence
-    applications in one integrated solution to allow better, and more dynamic,
-    consumer insights. Lily allows companies to process and analyze massive structured
-    and unstructured data, scale storage elastically and locate actionable data
-    quickly from large data sources in near real time.</dd>
-
-  <dt><a href="http://ning.com">Ning</a></dt>
-  <dd>Ning uses HBase to store and serve the results of processing user events
-    and log files, which allows us to provide near-real time analytics and
-    reporting. We use a small cluster of commodity machines with 4 cores and 16GB
-    of RAM per machine to handle all our analytics and reporting needs.</dd>
-
-  <dt><a href="http://www.worldcat.org">OCLC</a></dt>
-  <dd>OCLC uses HBase as the main data store for WorldCat, a union catalog which
-    aggregates the collections of 72,000 libraries in 112 countries and territories.
-    WorldCat is currently comprised of nearly 1 billion records with nearly 2
-    billion library ownership indications. We're running a 50 Node HBase cluster
-    and a separate offline map-reduce cluster.</dd>
-
-  <dt><a href="http://olex.openlogic.com">OpenLogic</a></dt>
-  <dd>OpenLogic stores all the world's Open Source packages, versions, files,
-    and lines of code in HBase for both near-real-time access and analytical
-    purposes. The production cluster has well over 100TB of disk spread across
-    nodes with 32GB+ RAM and dual-quad or dual-hex core CPU's.</dd>
-
-  <dt><a href="http://www.openplaces.org">Openplaces</a></dt>
-  <dd>Openplaces is a search engine for travel that uses HBase to store terabytes
-    of web pages and travel-related entity records (countries, cities, hotels,
-    etc.). We have dozens of MapReduce jobs that crunch data on a daily basis.
-    We use a 20-node cluster for development, a 40-node cluster for offline
-    production processing and an EC2 cluster for the live web site.</dd>
-
-  <dt><a href="http://www.pnl.gov">Pacific Northwest National Laboratory</a></dt>
-  <dd>Hadoop and HBase (Cloudera distribution) are being used within PNNL's
-    Computational Biology &amp; Bioinformatics Group for a systems biology data
-    warehouse project that integrates high throughput proteomics and transcriptomics
-    data sets coming from instruments in the Environmental  Molecular Sciences
-    Laboratory, a US Department of Energy national user facility located at PNNL.
-    The data sets are being merged and annotated with other public genomics
-    information in the data warehouse environment, with Hadoop analysis programs
-    operating on the annotated data in the HBase tables. This work is hosted by
-    <a href="http://www.pnl.gov/news/release.aspx?id=908">olympus</a>, a large PNNL
-    institutional computing cluster, with the HBase tables being stored in olympus's
-    Lustre file system.</dd>
-
-  <dt><a href="http://www.readpath.com/">ReadPath</a></dt>
-  <dd>|ReadPath uses HBase to store several hundred million RSS items and dictionary
-    for its RSS newsreader. Readpath is currently running on an 8 node cluster.</dd>
-
-  <dt><a href="http://resu.me/">resu.me</a></dt>
-  <dd>Career network for the net generation. We use HBase and Hadoop for all
-    aspects of our backend - user and resume data storage, analytics processing,
-    machine learning algorithms for our job recommendation engine. Our live
-    production site is directly served from HBase. We use cascading for running
-    offline data processing jobs.</dd>
-
-  <dt><a href="http://www.runa.com/">Runa Inc.</a></dt>
-  <dd>Runa Inc. offers a SaaS that enables online merchants to offer dynamic
-    per-consumer, per-product promotions embedded in their website. To implement
-    this we collect the click streams of all their visitors to determine along
-    with the rules of the merchant what promotion to offer the visitor at different
-    points of their browsing the Merchant website. So we have lots of data and have
-    to do lots of off-line and real-time analytics. HBase is the core for us.
-    We also use Clojure and our own open sourced distributed processing framework,
-    Swarmiji. The HBase Community has been key to our forward movement with HBase.
-    We're looking for experienced developers to join us to help make things go even
-    faster!</dd>
-
-  <dt><a href="http://www.sematext.com/">Sematext</a></dt>
-  <dd>Sematext runs
-    <a href="http://www.sematext.com/search-analytics/index.html">Search Analytics</a>,
-    a service that uses HBase to store search activity and MapReduce to produce
-    reports showing user search behaviour and experience. Sematext runs
-    <a href="http://www.sematext.com/spm/index.html">Scalable Performance Monitoring (SPM)</a>,
-    a service that uses HBase to store performance data over time, crunch it with
-    the help of MapReduce, and display it in a visually rich browser-based UI.
-    Interestingly, SPM features
-    <a href="http://www.sematext.com/spm/hbase-performance-monitoring/index.html">SPM for HBase</a>,
-    which is specifically designed to monitor all HBase performance metrics.</dd>
-
-  <dt><a href="http://www.socialmedia.com/">SocialMedia</a></dt>
-  <dd>SocialMedia uses HBase to store and process user events which allows us to
-    provide near-realtime user metrics and reporting. HBase forms the heart of
-    our Advertising Network data storage and management system. We use HBase as
-    a data source and sink for both realtime request cycle queries and as a
-    backend for mapreduce analysis.</dd>
-
-  <dt><a href="http://www.splicemachine.com/">Splice Machine</a></dt>
-  <dd>Splice Machine is built on top of HBase.  Splice Machine is a full-featured
-    ANSI SQL database that provides real-time updates, secondary indices, ACID
-    transactions, optimized joins, triggers, and UDFs.</dd>
-
-  <dt><a href="http://www.streamy.com/">Streamy</a></dt>
-  <dd>Streamy is a recently launched realtime social news site.  We use HBase
-    for all of our data storage, query, and analysis needs, replacing an existing
-    SQL-based system.  This includes hundreds of millions of documents, sparse
-    matrices, logs, and everything else once done in the relational system. We
-    perform significant in-memory caching of query results similar to a traditional
-    Memcached/SQL setup as well as other external components to perform joining
-    and sorting.  We also run thousands of daily MapReduce jobs using HBase tables
-    for log analysis, attention data processing, and feed crawling.  HBase has
-    helped us scale and distribute in ways we could not otherwise, and the
-    community has provided consistent and invaluable assistance.</dd>
-
-  <dt><a href="http://www.stumbleupon.com/">Stumbleupon</a></dt>
-  <dd>Stumbleupon and <a href="http://su.pr">Su.pr</a> use HBase as a real time
-    data storage and analytics platform. Serving directly out of HBase, various site
-    features and statistics are kept up to date in a real time fashion. We also
-    use HBase a map-reduce data source to overcome traditional query speed limits
-    in MySQL.</dd>
-
-  <dt><a href="http://www.tokenizer.org">Shopping Engine at Tokenizer</a></dt>
-  <dd>Shopping Engine at Tokenizer is a web crawler; it uses HBase to store URLs
-    and Outlinks (AnchorText + LinkedURL): more than a billion. It was initially
-    designed as Nutch-Hadoop extension, then (due to very specific 'shopping'
-    scenario) moved to SOLR + MySQL(InnoDB) (ten thousands queries per second),
-    and now - to HBase. HBase is significantly faster due to: no need for huge
-    transaction logs, column-oriented design exactly matches 'lazy' business logic,
-    data compression, !MapReduce support. Number of mutable 'indexes' (term from
-    RDBMS) significantly reduced due to the fact that each 'row::column' structure
-    is physically sorted by 'row'. MySQL InnoDB engine is best DB choice for
-    highly-concurrent updates. However, necessity to flash a block of data to
-    harddrive even if we changed only few bytes is obvious bottleneck. HBase
-    greatly helps: not-so-popular in modern DBMS 'delete-insert', 'mutable primary
-    key', and 'natural primary key' patterns become a big advantage with HBase.</dd>
-
-  <dt><a href="http://traackr.com/">Traackr</a></dt>
-  <dd>Traackr uses HBase to store and serve online influencer data in real-time.
-    We use MapReduce to frequently re-score our entire data set as we keep updating
-    influencer metrics on a daily basis.</dd>
-
-  <dt><a href="http://trendmicro.com/">Trend Micro</a></dt>
-  <dd>Trend Micro uses HBase as a foundation for cloud scale storage for a variety
-    of applications. We have been developing with HBase since version 0.1 and
-    production since version 0.20.0.</dd>
-
-  <dt><a href="http://www.twitter.com">Twitter</a></dt>
-  <dd>Twitter runs HBase across its entire Hadoop cluster. HBase provides a
-    distributed, read/write backup of all  mysql tables in Twitter's production
-    backend, allowing engineers to run MapReduce jobs over the data while maintaining
-    the ability to apply periodic row updates (something that is more difficult
-    to do with vanilla HDFS).  A number of applications including people search
-    rely on HBase internally for data generation. Additionally, the operations
-    team uses HBase as a timeseries database for cluster-wide monitoring/performance
-    data.</dd>
-
-  <dt><a href="http://www.udanax.org">Udanax.org</a></dt>
-  <dd>Udanax.org is a URL shortener which use 10 nodes HBase cluster to store URLs,
-    Web Log data and response the real-time request on its Web Server. This
-    application is now used for some twitter clients and a number of web sites.
-    Currently API requests are almost 30 per second and web redirection requests
-    are about 300 per second.</dd>
-
-  <dt><a href="http://www.veoh.com/">Veoh Networks</a></dt>
-  <dd>Veoh Networks uses HBase to store and process visitor (human) and entity
-    (non-human) profiles which are used for behavioral targeting, demographic
-    detection, and personalization services.  Our site reads this data in
-    real-time (heavily cached) and submits updates via various batch map/reduce
-    jobs. With 25 million unique visitors a month storing this data in a traditional
-    RDBMS is not an option. We currently have a 24 node Hadoop/HBase cluster and
-    our profiling system is sharing this cluster with our other Hadoop data
-    pipeline processes.</dd>
-
-  <dt><a href="http://www.videosurf.com/">VideoSurf</a></dt>
-  <dd>VideoSurf - "The video search engine that has taught computers to see".
-    We're using HBase to persist various large graphs of data and other statistics.
-    HBase was a real win for us because it let us store substantially larger
-    datasets without the need for manually partitioning the data and its
-    column-oriented nature allowed us to create schemas that were substantially
-    more efficient for storing and retrieving data.</dd>
-
-  <dt><a href="http://www.visibletechnologies.com/">Visible Technologies</a></dt>
-  <dd>Visible Technologies uses Hadoop, HBase, Katta, and more to collect, parse,
-    store, and search hundreds of millions of Social Media content. We get incredibly
-    fast throughput and very low latency on commodity hardware. HBase enables our
-    business to exist.</dd>
-
-  <dt><a href="http://www.worldlingo.com/">WorldLingo</a></dt>
-  <dd>The WorldLingo Multilingual Archive. We use HBase to store millions of
-    documents that we scan using Map/Reduce jobs to machine translate them into
-    all or selected target languages from our set of available machine translation
-    languages. We currently store 12 million documents but plan to eventually
-    reach the 450 million mark. HBase allows us to scale out as we need to grow
-    our storage capacities. Combined with Hadoop to keep the data replicated and
-    therefore fail-safe we have the backbone our service can rely on now and in
-    the future. !WorldLingo is using HBase since December 2007 and is along with
-    a few others one of the longest running HBase installation. Currently we are
-    running the latest HBase 0.20 and serving directly from it at
-    <a href="http://www.worldlingo.com/ma/enwiki/en/HBase">MultilingualArchive</a>.</dd>
-
-  <dt><a href="http://www.yahoo.com/">Yahoo!</a></dt>
-  <dd>Yahoo! uses HBase to store document fingerprint for detecting near-duplications.
-    We have a cluster of few nodes that runs HDFS, mapreduce, and HBase. The table
-    contains millions of rows. We use this for querying duplicated documents with
-    realtime traffic.</dd>
-
-  <dt><a href="http://h50146.www5.hp.com/products/software/security/icewall/eng/">HP IceWall SSO</a></dt>
-  <dd>HP IceWall SSO is a web-based single sign-on solution and uses HBase to store
-    user data to authenticate users. We have supported RDB and LDAP previously but
-    have newly supported HBase with a view to authenticate over tens of millions
-    of users and devices.</dd>
-
-  <dt><a href="http://www.ymc.ch/en/big-data-analytics-en?utm_source=hadoopwiki&amp;utm_medium=poweredbypage&amp;utm_campaign=ymc.ch">YMC AG</a></dt>
-  <dd><ul>
-    <li>operating a Cloudera Hadoop/HBase cluster for media monitoring purpose</li>
-    <li>offering technical and operative consulting for the Hadoop stack + ecosystem</li>
-    <li>editor of <a href="http://www.ymc.ch/en/hbase-split-visualisation-introducing-hannibal?utm_source=hadoopwiki&amp;utm_medium=poweredbypageamp;utm_campaign=ymc.ch">Hannibal</a>, a open-source tool
-    to visualize HBase regions sizes and splits that helps running HBase in production</li>
-  </ul></dd>
-  </dl>
-</section>
-</body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/pseudo-distributed.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/pseudo-distributed.xml b/src/main/site/xdoc/pseudo-distributed.xml
deleted file mode 100644
index 670f1e7..0000000
--- a/src/main/site/xdoc/pseudo-distributed.xml
+++ /dev/null
@@ -1,42 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title> 
-Running Apache HBase (TM) in pseudo-distributed mode
-    </title>
-  </properties>
-
-  <body>
-      <p>This page has been retired.  The contents have been moved to the 
-      <a href="http://hbase.apache.org/book.html#distributed">Distributed Operation: Pseudo- and Fully-distributed modes</a> section
- in the Reference Guide.
- </p>
-
- </body>
-
-</document>
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/replication.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/replication.xml b/src/main/site/xdoc/replication.xml
deleted file mode 100644
index a2fcfcb..0000000
--- a/src/main/site/xdoc/replication.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-
-<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
-          "http://forrest.apache.org/dtd/document-v20.dtd">
-
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>
-      Apache HBase (TM) Replication
-    </title>
-  </properties>
-  <body>
-    <p>This information has been moved to <a href="http://hbase.apache.org/book.html#cluster_replication">the Cluster Replication</a> section of the <a href="http://hbase.apache.org/book.html">Apache HBase Reference Guide</a>.</p>
-  </body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/resources.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/resources.xml b/src/main/site/xdoc/resources.xml
deleted file mode 100644
index 19548b6..0000000
--- a/src/main/site/xdoc/resources.xml
+++ /dev/null
@@ -1,45 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Other Apache HBase (TM) Resources</title>
-  </properties>
-
-<body>
-<section name="Other Apache HBase Resources">
-<section name="Books">
-<section name="HBase: The Definitive Guide">
-<p><a href="http://shop.oreilly.com/product/0636920014348.do">HBase: The Definitive Guide <i>Random Access to Your Planet-Size Data</i></a> by Lars George. Publisher: O'Reilly Media, Released: August 2011, Pages: 556.</p>
-</section>
-<section name="HBase In Action">
-<p><a href="http://www.manning.com/dimidukkhurana/">HBase In Action</a> By Nick Dimiduk and Amandeep Khurana.  Publisher: Manning, MEAP Began: January 2012, Softbound print: Fall 2012, Pages: 350.</p>
-</section>
-<section name="HBase Administration Cookbook">
-<p><a href="http://www.packtpub.com/hbase-administration-for-optimum-database-performance-cookbook/book">HBase Administration Cookbook</a> by Yifeng Jiang.  Publisher: PACKT Publishing, Release: Expected August 2012, Pages: 335.</p>
-</section>
-<section name="HBase High Performance Cookbook">
-  <p><a href="https://www.packtpub.com/big-data-and-business-intelligence/hbase-high-performance-cookbook">HBase High Performance Cookbook</a> by Ruchir Choudhry.  Publisher: PACKT Publishing, Release: January 2017, Pages: 350.</p>
-</section>
-</section>
-</section>
-</body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/sponsors.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/sponsors.xml b/src/main/site/xdoc/sponsors.xml
deleted file mode 100644
index 332f56a..0000000
--- a/src/main/site/xdoc/sponsors.xml
+++ /dev/null
@@ -1,50 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Apache HBase&#153; Sponsors</title>
-  </properties>
-
-<body>
-<section name="Sponsors">
-    <p>First off, thanks to <a href="http://www.apache.org/foundation/thanks.html">all who sponsor</a>
-       our parent, the Apache Software Foundation.
-    </p>
-<p>The below companies have been gracious enough to provide their commerical tool offerings free of charge to the Apache HBase&#153; project.
-<ul>
-	<li>The crew at <a href="http://www.ej-technologies.com/">ej-technologies</a> have
-        been let us use <a href="http://www.ej-technologies.com/products/jprofiler/overview.html">JProfiler</a> for years now.</li>
-	<li>The lads at <a href="http://headwaysoftware.com/">headway software</a> have
-        given us a license for <a href="http://headwaysoftware.com/products/?code=Restructure101">Restructure101</a>
-        so we can untangle our interdependency mess.</li>
-	<li><a href="http://www.yourkit.com">YourKit</a> allows us to use their <a href="http://www.yourkit.com/overview/index.jsp">Java Profiler</a>.</li>
-	<li>Some of us use <a href="http://www.jetbrains.com/idea">IntelliJ IDEA</a> thanks to <a href="http://www.jetbrains.com/">JetBrains</a>.</li>
-  <li>Thank you to Boris at <a href="http://www.vectorportal.com/">Vector Portal</a> for granting us a license on the <a href="http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp">image</a> on which our logo is based.</li>
-</ul>
-</p>
-</section>
-<section name="Sponsoring the Apache Software Foundation">
-<p>To contribute to the Apache Software Foundation, a good idea in our opinion, see the <a href="http://www.apache.org/foundation/sponsorship.html">ASF Sponsorship</a> page.
-</p>
-</section>
-</body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/main/site/xdoc/supportingprojects.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/supportingprojects.xml b/src/main/site/xdoc/supportingprojects.xml
deleted file mode 100644
index f949a57..0000000
--- a/src/main/site/xdoc/supportingprojects.xml
+++ /dev/null
@@ -1,161 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<document xmlns="http://maven.apache.org/XDOC/2.0"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
-  <properties>
-    <title>Supporting Projects</title>
-  </properties>
-
-<body>
-<section name="Supporting Projects">
-  <p>This page is a list of projects that are related to HBase. To
-    have your project added, file a documentation JIRA or email
-    <a href="mailto:dev@hbase.apache.org">hbase-dev</a> with the relevant
-    information. If you notice out-of-date information, use the same avenues to
-    report it.
-  </p>
-  <p><b>These items are user-submitted and the HBase team assumes no responsibility for their accuracy.</b></p>
-  <h3>Projects that add new features to HBase</h3>
-  <dl>
-   <dt><a href="https://github.com/XiaoMi/themis/">Themis</a></dt>
-   <dd>Themis provides cross-row/cross-table transaction on HBase based on
-    Google's Percolator.</dd>
-   <dt><a href="https://github.com/caskdata/tephra">Tephra</a></dt>
-   <dd>Cask Tephra provides globally consistent transactions on top of Apache
-    HBase.</dd>
-   <dt><a href="https://github.com/VCNC/haeinsa">Haeinsa</a></dt>
-   <dd>Haeinsa is linearly scalable multi-row, multi-table transaction library
-    for HBase.</dd>
-   <dt><a href="https://github.com/juwi/HBase-TAggregator">HBase TAggregator</a></dt>
-   <dd>An HBase coprocessor for timeseries-based aggregations.</dd>
-   <dt><a href="http://trafodion.incubator.apache.org/">Apache Trafodion</a></dt>
-   <dd>Apache Trafodion is a webscale SQL-on-Hadoop solution enabling
-    transactional or operational workloads on Hadoop.</dd>
-   <dt><a href="http://phoenix.apache.org/">Apache Phoenix</a></dt>
-   <dd>Apache Phoenix is a relational database layer over HBase delivered as a
-    client-embedded JDBC driver targeting low latency queries over HBase data.</dd>
-   <dt><a href="https://github.com/cloudera/hue/tree/master/apps/hbase">Hue HBase Browser</a></dt>
-   <dd>An Easy &amp; Powerful WebUI for HBase, distributed with <a href="https://www.gethue.com">Hue</a>.</dd>
-   <dt><a href="https://github.com/NGDATA/hbase-indexer/tree/master/hbase-sep">HBase SEP</a></dt>
-   <dd>the HBase Side Effect Processor, a system for asynchronously and reliably listening to HBase
-    mutation events, based on HBase replication.</dd>
-   <dt><a href="https://github.com/ngdata/hbase-indexer">Lily HBase Indexer</a></dt>
-   <dd>indexes HBase content to Solr by listening to the replication stream
-    (uses the HBase SEP).</dd>
-   <dt><a href="https://github.com/sonalgoyal/crux/">Crux</a></dt>
-   <dd> - HBase Reporting and Analysis with support for simple and composite keys,
-    get and range scans, column based filtering, charting.</dd>
-   <dt><a href="https://github.com/yahoo/omid/">Omid</a></dt>
-   <dd> - Lock-free transactional support on top of HBase providing Snapshot
-    Isolation.</dd>
-   <dt><a href="http://dev.tailsweep.com/projects/parhely">Parhely</a></dt>
-   <dd>ORM for HBase</dd>
-   <dt><a href="http://code.google.com/p/hbase-writer/">HBase-Writer</a></dt>
-   <dd> Heritrix2 Processor for writing crawls to HBase.</dd>
-   <dt><a href="http://www.pigi-project.org/">Pigi Project</a></dt>
-   <dd>The Pigi Project is an ORM-like framework. It includes a configurable
-    index system and a simple object to HBase mapping framework (or indexing for
-    HBase if you like).  Designed for use by web applications.</dd>
-   <dt><a href="http://code.google.com/p/hbase-thrift/">hbase-thrift</a></dt>
-   <dd>hbase-thrift generates and installs Perl and Python Thrift bindings for
-    HBase.</dd>
-   <dt><a href="http://belowdeck.kissintelligentsystems.com/ohm">OHM</a></dt>
-   <dd>OHM is a weakly relational ORM for HBase which provides Object Mapping and
-    Column indexing. It has its own compiler capable of generating interface
-    code for multiple languages. Currently C# (via the Thrift API), with support
-    for Java currently in development. The compiler is easily extensible to add
-    support for other languages.</dd>
-   <dt><a href="http://datastore.googlecode.com">datastore</a></dt>
-   <dd>Aims to be an implementation of the
-    <a href="http://code.google.com/appengine/docs/python/datastore/">Google app-engine datastore</a>
-    in Java using HBase instead of bigtable.</dd>
-   <dt><a href="http://datanucleus.org">DataNucleus</a></dt>
-   <dd>DataNucleus is a Java JDO/JPA/REST implementation. It supports HBase and
-    many other datastores.</dd>
-   <dt><a href="http://github.com/impetus-opensource/Kundera">Kundera</a></dt>
-   <dd>Kundera is a JPA 2.0 based object-datastore mapping library for HBase,
-    Cassandra and MongoDB.</dd>
-   <dt><a href="http://github.com/zohmg/zohmg/tree/master">Zohmg</a></dt>
-   <dd>Zohmg is a time-series data store that uses HBase as its backing store.</dd>
-   <dt><a href="http://grails.org/plugin/gorm-hbase">Grails Support</a></dt>
-   <dd>Grails HBase plug-in.</dd>
-   <dt><a href="http://www.bigrecord.org">BigRecord</a></dt>
-   <dd>is an active_record-based object mapping layer for Ruby on Rails.</dd>
-   <dt><a href="http://github.com/greglu/hbase-stargate">hbase-stargate</a></dt>
-   <dd>Ruby client for HBase Stargate.</dd>
-   <dt><a href="http://github.com/ghelmling/meetup.beeno">Meetup.Beeno</a></dt>
-   <dd>Meetup.Beeno is a simple HBase Java "beans" mapping framework based on
-    annotations. It includes a rudimentary high level query API that generates
-    the appropriate server-side filters.</dd>
-   <dt><a href="http://www.springsource.org/spring-data/hadoop">Spring Hadoop</a></dt>
-   <dd> - The Spring Hadoop project provides support for writing Apache Hadoop
-    applications that benefit from the features of Spring, Spring Batch and
-    Spring Integration.</dd>
-   <dt><a href="https://jira.springsource.org/browse/SPR-5950">Spring Framework HBase Template</a></dt>
-   <dd>Spring Framework HBase Template provides HBase data access templates
-    similar to what is provided in Spring for JDBC, Hibernate, iBatis, etc.
-    If you find this useful, please vote for its inclusion in the Spring Framework.</dd>
-   <dt><a href="http://github.com/davidsantiago/clojure-hbase">Clojure-HBase</a></dt>
-   <dd>A library for convenient access to HBase from Clojure.</dd>
-   <dt><a href="http://www.lilyproject.org/lily/about/playground/hbaseindexes.html">HBase indexing library</a></dt>
-   <dd>A library for building and querying HBase-table-based indexes.</dd>
-   <dt><a href="http://github.com/akkumar/hbasene">HBasene</a></dt>
-   <dd>Lucene+HBase - Using HBase as the backing store for the TF-IDF
-    representations needed by Lucene. Also, contains a library for constructing
-    lucene indices from HBase schema.</dd>
-   <dt><a href="http://github.com/larsgeorge/jmxtoolkit">JMXToolkit</a></dt>
-   <dd>A HBase tailored JMX toolkit enabling monitoring with Cacti and checking
-    with Nagios or similar.</dd>
-   <dt><a href="http://github.com/ykulbak/ihbase">IHBASE</a></dt>
-   <dd>IHBASE provides faster scans by indexing regions, each region has its own
-    index. The indexed columns are user-defined and indexes can be intersected or
-    joined in a single query.</dd>
-   <dt><a href="http://github.com/apurtell/hbase-ec2">HBASE EC2 scripts</a></dt>
-   <dd>This collection of bash scripts allows you to run HBase clusters on
-    Amazon's Elastic Compute Cloud (EC2) service with best practices baked in.</dd>
-   <dt><a href="http://github.com/apurtell/hbase-stargate">Stargate</a></dt>
-   <dd>Stargate provides an enhanced RESTful interface.</dd>
-   <dt><a href="http://github.com/hbase-trx/hbase-transactional-tableindexed">HBase-trx</a></dt>
-   <dd>HBase-trx provides Transactional (JTA) and indexed extensions of HBase.</dd>
-   <dt><a href="http://github.com/simplegeo/python-hbase-thrift">HBase Thrift Python client Debian package</a></dt>
-   <dd>Debian packages for the HBase Thrift Python client (see readme for
-    sources.list setup)</dd>
-   <dt><a href="http://github.com/amitrathore/capjure">capjure</a></dt>
-   <dd>capjure is a persistence helper for HBase. It is written in the Clojure
-    language, and supports persisting of native hash-maps.</dd>
-   <dt><a href="http://github.com/sematext/HBaseHUT">HBaseHUT</a></dt>
-   <dd>(High Update Throughput for HBase) It focuses on write performance during
-    records update (by avoiding doing Get on every Put to update record).</dd>
-   <dt><a href="http://github.com/sematext/HBaseWD">HBaseWD</a></dt>
-   <dd>HBase Writes Distributor spreads records over the cluster even when their
-    keys are sequential, while still allowing fast range scans over them</dd>
-   <dt><a href="http://code.google.com/p/hbase-jdo/">HBase UI Tool &amp; Util</a></dt>
-   <dd>HBase UI Tool &amp; Util is an HBase UI client and simple util module.
-    It can handle hbase more easily like jdo(not persistence api)</dd>
-  </dl>
-  <h3>Example HBase Applications</h3>
-  <ul>
-    <li><a href="http://github.com/andreisavu/feedaggregator">HBase powered feed aggregator</a>
-    by Savu Andrei -- 200909</li>
-  </ul>
-</section>
-</body>
-</document>

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/acid-semantics.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/acid-semantics.adoc b/src/site/asciidoc/acid-semantics.adoc
new file mode 100644
index 0000000..0038901
--- /dev/null
+++ b/src/site/asciidoc/acid-semantics.adoc
@@ -0,0 +1,118 @@
+////
+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.
+////
+
+= Apache HBase (TM) ACID Properties
+
+== About this Document
+
+Apache HBase (TM) is not an ACID compliant database. However, it does guarantee certain specific properties.
+
+This specification enumerates the ACID properties of HBase.
+
+== Definitions
+
+For the sake of common vocabulary, we define the following terms:
+Atomicity::
+  An operation is atomic if it either completes entirely or not at all.
+
+Consistency::
+  All actions cause the table to transition from one valid state directly to another (eg a row will not disappear during an update, etc).
+
+Isolation::
+  an operation is isolated if it appears to complete independently of any other concurrent transaction.
+
+Durability::
+  Any update that reports &quot;successful&quot; to the client will not be lost.
+
+Visibility::
+  An update is considered visible if any subsequent read will see the update as having been committed.
+
+
+The terms _must_ and _may_ are used as specified by link:[RFC 2119].
+
+In short, the word &quot;must&quot; implies that, if some case exists where the statement is not true, it is a bug. The word _may_ implies that, even if the guarantee is provided in a current release, users should not rely on it.
+
+== APIs to Consider
+- Read APIs
+* get
+* scan
+- Write APIs
+* put
+* batch put
+* delete
+- Combination (read-modify-write) APIs
+* incrementColumnValue
+* checkAndPut
+
+== Guarantees Provided
+
+.Atomicity
+.  All mutations are atomic within a row. Any put will either wholely succeed or wholely fail.footnoteref[Puts will either wholely succeed or wholely fail, provided that they are actually sent to the RegionServer.  If the writebuffer is used, Puts will not be sent until the writebuffer is filled or it is explicitly flushed.]
+.. An operation that returns a _success_ code has completely succeeded.
+.. An operation that returns a _failure_ code has completely failed.
+.. An operation that times out may have succeeded and may have failed. However, it will not have partially succeeded or failed.
+. This is true even if the mutation crosses multiple column families within a row.
+. APIs that mutate several rows will _not_ be atomic across the multiple rows. For example, a multiput that operates on rows 'a','b', and 'c' may return having mutated some but not all of the rows. In such cases, these APIs will return a list of success codes, each of which may be succeeded, failed, or timed out as described above.
+. The checkAndPut API happens atomically like the typical _compareAndSet (CAS)_ operation found in many hardware architectures.
+. The order of mutations is seen to happen in a well-defined order for each row, with no interleaving. For example, if one writer issues the mutation `a=1,b=1,c=1` and another writer issues the mutation `a=2,b=2,c=`, the row must either be `a=1,b=1,c=1` or `a=2,b=2,c=2` and must *not* be something like `a=1,b=2,c=1`. +
+NOTE:This is not true _across rows_ for multirow batch mutations.
+
+== Consistency and Isolation
+. All rows returned via any access API will consist of a complete row that existed at some point in the table's history.
+. This is true across column families - i.e a get of a full row that occurs concurrent with some mutations 1,2,3,4,5 will return a complete row that existed at some point in time between mutation i and i+1 for some i between 1 and 5.
+. The state of a row will only move forward through the history of edits to it.
+
+== Consistency of Scans
+A scan is *not* a consistent view of a table. Scans do *not* exhibit _snapshot isolation_.
+
+Rather, scans have the following properties:
+. Any row returned by the scan will be a consistent view (i.e. that version of the complete row existed at some point in time)footnoteref[consistency,A consistent view is not guaranteed intra-row scanning -- i.e. fetching a portion of a row in one RPC then going back to fetch another portion of the row in a subsequent RPC. Intra-row scanning happens when you set a limit on how many values to return per Scan#next (See link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html#setBatch(int)"[Scan#setBatch(int)]).]
+. A scan will always reflect a view of the data _at least as new as_ the beginning of the scan. This satisfies the visibility guarantees enumerated below.
+.. For example, if client A writes data X and then communicates via a side channel to client B, any scans started by client B will contain data at least as new as X.
+.. A scan _must_ reflect all mutations committed prior to the construction of the scanner, and _may_ reflect some mutations committed subsequent to the construction of the scanner.
+.. Scans must include _all_ data written prior to the scan (except in the case where data is subsequently mutated, in which case it _may_ reflect the mutation)
+
+Those familiar with relational databases will recognize this isolation level as "read committed".
+
+NOTE: The guarantees listed above regarding scanner consistency are referring to "transaction commit time", not the "timestamp" field of each cell. That is to say, a scanner started at time _t_ may see edits with a timestamp value greater than _t_, if those edits were committed with a "forward dated" timestamp before the scanner was constructed.
+
+== Visibility
+
+. When a client receives a &quot;success&quot; response for any mutation, that mutation is immediately visible to both that client and any client with whom it later communicates through side channels.footnoteref[consistency]
+. A row must never exhibit so-called "time-travel" properties. That is to say, if a series of mutations moves a row sequentially through a series of states, any sequence of concurrent reads will return a subsequence of those states. +
+For example, if a row's cells are mutated using the `incrementColumnValue` API, a client must never see the value of any cell decrease. +
+This is true regardless of which read API is used to read back the mutation.
+. Any version of a cell that has been returned to a read operation is guaranteed to be durably stored.
+
+== Durability
+. All visible data is also durable data. That is to say, a read will never return data that has not been made durable on disk.footnoteref[durability,In the context of Apache HBase, _durably on disk_; implies an `hflush()` call on the transaction log. This does not actually imply an `fsync()` to magnetic media, but rather just that the data has been written to the OS cache on all replicas of the log. In the case of a full datacenter power loss, it is possible that the edits are not truly durable.]
+. Any operation that returns a &quot;success&quot; code (eg does not throw an exception) will be made durable.footnoteref[durability]
+. Any operation that returns a &quot;failure&quot; code will not be made durable (subject to the Atomicity guarantees above).
+. All reasonable failure scenarios will not affect any of the guarantees of this document.
+
+== Tunability
+
+All of the above guarantees must be possible within Apache HBase. For users who would like to trade off some guarantees for performance, HBase may offer several tuning options. For example:
+
+* Visibility may be tuned on a per-read basis to allow stale reads or time travel.
+* Durability may be tuned to only flush data to disk on a periodic basis.
+
+== More Information
+
+For more information, see the link:book.html#client[client architecture] and  link:book.html#datamodel[data model] sections in the Apache HBase Reference Guide. 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/bulk-loads.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/bulk-loads.adoc b/src/site/asciidoc/bulk-loads.adoc
new file mode 100644
index 0000000..fc320d8
--- /dev/null
+++ b/src/site/asciidoc/bulk-loads.adoc
@@ -0,0 +1,23 @@
+////
+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.
+////
+
+= Bulk Loads in Apache HBase (TM)
+
+This page has been retired.  The contents have been moved to the link:book.html#arch.bulk.load[Bulk Loading] section in the Reference Guide.
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/cygwin.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/cygwin.adoc b/src/site/asciidoc/cygwin.adoc
new file mode 100644
index 0000000..11c4df4
--- /dev/null
+++ b/src/site/asciidoc/cygwin.adoc
@@ -0,0 +1,197 @@
+////
+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.
+////
+
+
+== Installing Apache HBase (TM) on Windows using Cygwin
+
+== Introduction
+
+link:http://hbase.apache.org[Apache HBase (TM)] is a distributed, column-oriented store, modeled after Google's link:http://research.google.com/archive/bigtable.html[BigTable]. Apache HBase is built on top of link:http://hadoop.apache.org[Hadoop] for its link:http://hadoop.apache.org/mapreduce[MapReduce] link:http://hadoop.apache.org/hdfs[distributed file system] implementations. All these projects are open-source and part of the link:http://www.apache.org[Apache Software Foundation].
+
+== Purpose
+
+This document explains the *intricacies* of running Apache HBase on Windows using Cygwin* as an all-in-one single-node installation for testing and development. The HBase link:http://hbase.apache.org/apidocs/overview-summary.html#overview_description[Overview] and link:book.html#getting_started[QuickStart] guides on the other hand go a long way in explaning how to setup link:http://hadoop.apache.org/hbase[HBase] in more complex deployment scenarios.
+
+== Installation
+
+For running Apache HBase on Windows, 3 technologies are required: 
+* Java
+* Cygwin
+* SSH 
+
+The following paragraphs detail the installation of each of the aforementioned technologies.
+
+=== Java
+
+HBase depends on the link:http://java.sun.com/javase/6/[Java Platform, Standard Edition, 6 Release]. So the target system has to be provided with at least the Java Runtime Environment (JRE); however if the system will also be used for development, the Jave Development Kit (JDK) is preferred. You can download the latest versions for both from link:http://java.sun.com/javase/downloads/index.jsp[Sun's download page]. Installation is a simple GUI wizard that guides you through the process.
+
+=== Cygwin
+
+Cygwin is probably the oddest technology in this solution stack. It provides a dynamic link library that emulates most of a *nix environment on Windows. On top of that a whole bunch of the most common *nix tools are supplied. Combined, the DLL with the tools form a very *nix-alike environment on Windows.
+
+For installation, Cygwin provides the link:http://cygwin.com/setup.exe[`setup.exe` utility] that tracks the versions of all installed components on the target system and provides the mechanism for installing or updating everything from the mirror sites of Cygwin.
+
+To support installation, the `setup.exe` utility uses 2 directories on the target system. The *Root* directory for Cygwin (defaults to _C:\cygwin)_ which will become _/_ within the eventual Cygwin installation; and the *Local Package* directory (e.g. _C:\cygsetup_ that is the cache where `setup.exe`stores the packages before they are installed. The cache must not be the same folder as the Cygwin root.
+
+Perform following steps to install Cygwin, which are elaboratly detailed in the link:http://cygwin.com/cygwin-ug-net/setup-net.html[2nd chapter] of the link:http://cygwin.com/cygwin-ug-net/cygwin-ug-net.html[Cygwin User's Guide].
+
+. Make sure you have `Administrator` privileges on the target system.
+. Choose and create you Root and *Local Package* directories. A good suggestion is to use `C:\cygwin\root` and `C:\cygwin\setup` folders.
+. Download the `setup.exe` utility and save it to the *Local Package* directory. Run the `setup.exe` utility.
+.. Choose  the `Install from Internet` option.
+.. Choose your *Root* and *Local Package* folders.
+.. Select an appropriate mirror.
+.. Don't select any additional packages yet, as we only want to install Cygwin for now.
+.. Wait for download and install.
+.. Finish the installation.
+. Optionally, you can now also add a shortcut to your Start menu pointing to the `setup.exe` utility in the *Local Package *folder.
+. Add `CYGWIN_HOME` system-wide environment variable that points to your *Root* directory.
+. Add `%CYGWIN_HOME%\bin` to the end of your `PATH` environment variable.
+. Reboot the sytem after making changes to the environment variables otherwise the OS will not be able to find the Cygwin utilities.
+. Test your installation by running your freshly created shortcuts or the `Cygwin.bat` command in the *Root* folder. You should end up in a terminal window that is running a link:http://www.gnu.org/software/bash/manual/bashref.html[Bash shell]. Test the shell by issuing following commands:
+.. `cd /` should take you to thr *Root* directory in Cygwin.
+.. The `LS` commands that should list all files and folders in the current directory.
+.. Use the `exit` command to end the terminal.
+. When needed, to *uninstall* Cygwin you can simply delete the *Root* and *Local Package* directory, and the *shortcuts* that were created during installation.
+
+=== SSH
+
+HBase (and Hadoop) rely on link:http://nl.wikipedia.org/wiki/Secure_Shell[*SSH*] for interprocess/-node *communication* and launching* remote commands*. SSH will be provisioned on the target system via Cygwin, which supports running Cygwin programs as *Windows services*!
+
+. Rerun the `*setup.exe*`* utility*.
+. Leave all parameters as is, skipping through the wizard using the `Next` button until the `Select Packages` panel is shown.
+. Maximize the window and click the `View` button to toggle to the list view, which is ordered alfabetically on `Package`, making it easier to find the packages we'll need.
+. Select the following packages by clicking the status word (normally `Skip`) so it's marked for installation. Use the `Next `button to download and install the packages.
+.. `OpenSSH`
+.. `tcp_wrappers`
+.. `diffutils`
+.. `zlib`
+. Wait for the install to complete and finish the installation.
+
+=== HBase
+
+Download the *latest release* of Apache HBase from link:http://www.apache.org/dyn/closer.cgi/hbase/. As the Apache HBase distributable is just a zipped archive, installation is as simple as unpacking the archive so it ends up in its final *installation* directory. Notice that HBase has to be installed in Cygwin and a good directory suggestion is to use `/usr/local/` (or [`*Root* directory]\usr\local` in Windows slang). You should end up with a `/usr/local/hbase-_versi` installation in Cygwin.
+
+This finishes installation. We go on with the configuration.
+
+== Configuration
+
+There are 3 parts left to configure: *Java, SSH and HBase* itself. Following paragraphs explain eacht topic in detail.
+
+=== Java
+
+One important thing to remember in shell scripting in general (i.e. *nix and Windows) is that managing, manipulating and assembling path names that contains spaces can be very hard, due to the need to escape and quote those characters and strings. So we try to stay away from spaces in path names. *nix environments can help us out here very easily by using *symbolic links*.
+
+. Create a link in `/usr/local` to the Java home directory by using the following command and substituting the name of your chosen Java environment: +
+----
+LN -s /cygdrive/c/Program\ Files/Java/*_jre name_*/usr/local/*_jre name_*
+----
+. Test your java installation by changing directories to your Java folder `CD /usr/local/_jre name_` and issueing the command `./bin/java -version`. This should output your version of the chosen JRE.
+
+=== SSH 
+
+Configuring *SSH *is quite elaborate, but primarily a question of launching it by default as a* Windows service*.
+
+. On Windows Vista and above make sure you run the Cygwin shell with *elevated privileges*, by right-clicking on the shortcut an using `Run as Administrator`.
+. First of all, we have to make sure the *rights on some crucial files* are correct. Use the commands underneath. You can verify all rights by using the `LS -L` command on the different files. Also, notice the auto-completion feature in the shell using `TAB` is extremely handy in these situations.
+.. `chmod +r /etc/passwd` to make the passwords file readable for all
+.. `chmod u+w /etc/passwd` to make the passwords file writable for the owner
+.. `chmod +r /etc/group` to make the groups file readable for all
+.. `chmod u+w /etc/group` to make the groups file writable for the owner
+.. `chmod 755 /var` to make the var folder writable to owner and readable and executable to all
+. Edit the */etc/hosts.allow* file using your favorite editor (why not VI in the shell!) and make sure the following two lines are in there before the `PARANOID` line: +
+----
+ALL : localhost 127.0.0.1/32 : allow
+ALL : [::1]/128 : allow
+----
+. Next we have to *configure SSH* by using the script `ssh-host-config`.
+.. If this script asks to overwrite an existing `/etc/ssh_config`, answer `yes`.
+.. If this script asks to overwrite an existing `/etc/sshd_config`, answer `yes`.
+.. If this script asks to use privilege separation, answer `yes`.
+.. If this script asks to install `sshd` as a service, answer `yes`. Make sure you started your shell as Adminstrator!
+.. If this script asks for the CYGWIN value, just `enter` as the default is `ntsec`.
+.. If this script asks to create the `sshd` account, answer `yes`.
+.. If this script asks to use a different user name as service account, answer `no` as the default will suffice.
+.. If this script asks to create the `cyg_server` account, answer `yes`. Enter a password for the account.
+. *Start the SSH service* using `net start sshd` or `cygrunsrv  --start  sshd`. Notice that `cygrunsrv` is the utility that make the process run as a Windows service. Confirm that you see a message stating that `the CYGWIN sshd service  was started succesfully.`
+. Harmonize Windows and Cygwin* user account* by using the commands: +
+----
+mkpasswd -cl > /etc/passwd
+mkgroup --local > /etc/group
+----
+. Test *the installation of SSH:
+.. Open a new Cygwin terminal.
+.. Use the command `whoami` to verify your userID.
+.. Issue an `ssh localhost` to connect to the system itself.
+.. Answer `yes` when presented with the server's fingerprint.
+.. Issue your password when prompted.
+.. Test a few commands in the remote session
+.. The `exit` command should take you back to your first shell in Cygwin.
+. `Exit` should terminate the Cygwin shell.
+
+=== HBase
+
+If all previous configurations are working properly, we just need some tinkering at the *HBase config* files to properly resolve on Windows/Cygwin. All files and paths referenced here start from the HBase `[*installation* directory]` as working directory.
+
+. HBase uses the `./conf/*hbase-env.sh*` to configure its dependencies on the runtime environment. Copy and uncomment following lines just underneath their original, change them to fit your environemnt. They should read something like: +
+----
+export JAVA_HOME=/usr/local/_jre name_
+export HBASE_IDENT_STRING=$HOSTNAME
+----
+. HBase uses the _./conf/`*hbase-default.xml*`_ file for configuration. Some properties do not resolve to existing directories because the JVM runs on Windows. This is the major issue to keep in mind when working with Cygwin: within the shell all paths are *nix-alike, hence relative to the root `/`. However, every parameter that is to be consumed within the windows processes themself, need to be Windows settings, hence `C:\`-alike. Change following propeties in the configuration file, adjusting paths where necessary to conform with your own installation:
+.. `hbase.rootdir` must read e.g. `file:///C:/cygwin/root/tmp/hbase/data`
+.. `hbase.tmp.dir` must read `C:/cygwin/root/tmp/hbase/tmp`
+.. `hbase.zookeeper.quorum` must read `127.0.0.1` because for some reason `localhost` doesn't seem to resolve properly on Cygwin.
+. Make sure the configured `hbase.rootdir` and `hbase.tmp.dir` *directories exist* and have the proper* rights* set up e.g. by issuing a `chmod 777` on them.
+
+== Testing
+
+This should conclude the installation and configuration of Apache HBase on Windows using Cygwin. So it's time *to test it*.
+
+. Start a Cygwin* terminal*, if you haven't already.
+. Change directory to HBase *installation* using `CD /usr/local/hbase-_version_`, preferably using auto-completion.
+. *Start HBase* using the command `./bin/start-hbase.sh`
+.. When prompted to accept the SSH fingerprint, answer `yes`.
+.. When prompted, provide your password. Maybe multiple times.
+.. When the command completes, the HBase server should have started.
+.. However, to be absolutely certain, check the logs in the `./logs` directory for any exceptions.
+. Next we *start the HBase shell* using the command `./bin/hbase shell`
+. We run some simple *test commands*
+.. Create a simple table using command `create 'test', 'data'`
+.. Verify the table exists using the command `list`
+.. Insert data into the table using e.g. +
+----
+put 'test', 'row1', 'data:1', 'value1'
+put 'test', 'row2', 'data:2', 'value2'
+put 'test', 'row3', 'data:3', 'value3'
+----
+.. List all rows in the table using the command `scan 'test'` that should list all the rows previously inserted. Notice how 3 new columns where added without changing the schema!
+.. Finally we get rid of the table by issuing `disable 'test'` followed by `drop 'test'` and verified by `list` which should give an empty listing.
+. *Leave the shell* by `exit`
+. To *stop the HBase server* issue the `./bin/stop-hbase.sh` command. And wait for it to complete!!! Killing the process might corrupt your data on disk.
+. In case of *problems*,
+.. Verify the HBase logs in the `./logs` directory.
+.. Try to fix the problem
+.. Get help on the forums or IRC (`#hbase@freenode.net`). People are very active and keen to help out!
+.. Stop and retest the server.
+
+== Conclusion
+
+Now your *HBase *server is running, *start coding* and build that next killer app on this particular, but scalable datastore!
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7da47509/src/site/asciidoc/export_control.adoc
----------------------------------------------------------------------
diff --git a/src/site/asciidoc/export_control.adoc b/src/site/asciidoc/export_control.adoc
new file mode 100644
index 0000000..1bbefb5
--- /dev/null
+++ b/src/site/asciidoc/export_control.adoc
@@ -0,0 +1,44 @@
+////
+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.
+////
+
+
+= Export Control
+
+This distribution uses or includes cryptographic software. The country in
+which you currently reside may have restrictions on the import, possession,
+use, and/or re-export to another country, of encryption software. BEFORE
+using any encryption software, please check your country's laws, regulations
+and policies concerning the import, possession, or use, and re-export of
+encryption software, to see if this is permitted. See the
+link:http://www.wassenaar.org/[Wassenaar Arrangement] for more
+information.
+
+The U.S. Government Department of Commerce, Bureau of Industry and Security 
+(BIS), has classified this software as Export Commodity Control Number (ECCN) 
+5D002.C.1, which includes information security software using or performing 
+cryptographic functions with asymmetric algorithms. The form and manner of this
+Apache Software Foundation distribution makes it eligible for export under the 
+License Exception ENC Technology Software Unrestricted (TSU) exception (see the
+BIS Export Administration Regulations, Section 740.13) for both object code and
+source code.
+
+Apache HBase uses the built-in java cryptography libraries. See Oracle's
+information regarding
+link:http://www.oracle.com/us/products/export/export-regulations-345813.html[Java cryptographic export regulations]
+for more details.
\ No newline at end of file

[24/27] hbase git commit: Revert "HBASE-18511 Default no regions on master"

Posted by bu...@apache.org.
Revert "HBASE-18511 Default no regions on master"

This reverts commit a8e0267c00409da7e3e03cfb34168787873ef075.


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

Branch: refs/heads/HBASE-18467
Commit: 03390684cc8967f6e733932bb03d33553c6c859a
Parents: b653d1b
Author: Michael Stack <st...@apache.org>
Authored: Tue Aug 8 13:37:56 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 8 13:37:56 2017 +0800

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/03390684/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 445f80f..8f5b6f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -1111,8 +1111,12 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
 
-  // Master carries no regions by default.
-  private static final String[] DEFAULT_TABLES_ON_MASTER = new String[] {};
+  // Regions of these tables are put on the master by default.
+  private static final String[] DEFAULT_TABLES_ON_MASTER =
+    new String[] {AccessControlLists.ACL_TABLE_NAME.getNameAsString(),
+      TableName.NAMESPACE_TABLE_NAME.getNameAsString(),
+      TableName.META_TABLE_NAME.getNameAsString()};
+
   public static final String TABLES_ON_MASTER =
     "hbase.balancer.tablesOnMaster";
 
@@ -1123,18 +1127,18 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   protected MasterServices services;
 
   /**
-   * By default, master carries no regions. This method returns null.
-   * If you want master to carry system tables say, then set
-   * TABLES_ON_MASTER to AccessControlLists.ACL_TABLE_NAME.getNameAsString(),
-   * TableName.NAMESPACE_TABLE_NAME.getNameAsString(),
-   * TableName.META_TABLE_NAME.getNameAsString()
+   * By default, regions of some small system tables such as meta,
+   * namespace, and acl are assigned to the active master. If you don't
+   * want to assign any region to the active master, you need to
+   * configure "hbase.balancer.tablesOnMaster" to "none".
    */
   protected static String[] getTablesOnMaster(Configuration conf) {
     String valueString = conf.get(TABLES_ON_MASTER);
     if (valueString == null) {
       return DEFAULT_TABLES_ON_MASTER;
     }
-    if (valueString == null || valueString.trim().isEmpty()) {
+    valueString = valueString.trim();
+    if (valueString.equalsIgnoreCase("none")) {
       return null;
     }
     return StringUtils.getStrings(valueString);


[23/27] hbase git commit: HBASE-18517 limit max log message width in log4j

Posted by bu...@apache.org.
HBASE-18517 limit max log message width in log4j

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: b653d1bf054737ea841a9298f4eb58fb1a00f1be
Parents: a8e0267
Author: Vikas Vishwakarma <vv...@salesforce.com>
Authored: Sat Aug 5 15:14:09 2017 +0530
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 8 12:11:13 2017 +0800

----------------------------------------------------------------------
 conf/log4j.properties | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b653d1bf/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/conf/log4j.properties b/conf/log4j.properties
index a6a560e..15545ff 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -40,7 +40,7 @@ log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
 log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
 
 # Pattern format: Date LogLevel LoggerName LogMessage
-log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %.1000m%n
 
 # Rolling File Appender properties
 hbase.log.maxfilesize=256MB
@@ -54,7 +54,7 @@ log4j.appender.RFA.MaxFileSize=${hbase.log.maxfilesize}
 log4j.appender.RFA.MaxBackupIndex=${hbase.log.maxbackupindex}
 
 log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
-log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %.1000m%n
 
 #
 # Security audit appender
@@ -67,7 +67,7 @@ log4j.appender.RFAS.File=${hbase.log.dir}/${hbase.security.log.file}
 log4j.appender.RFAS.MaxFileSize=${hbase.security.log.maxfilesize}
 log4j.appender.RFAS.MaxBackupIndex=${hbase.security.log.maxbackupindex}
 log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
-log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %.1000m%n
 log4j.category.SecurityLogger=${hbase.security.logger}
 log4j.additivity.SecurityLogger=false
 #log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.access.AccessController=TRACE
@@ -85,7 +85,7 @@ log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
 log4j.appender.console=org.apache.log4j.ConsoleAppender
 log4j.appender.console.target=System.err
 log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %.1000m%n
 
 log4j.appender.asyncconsole=org.apache.hadoop.hbase.AsyncConsoleAppender
 log4j.appender.asyncconsole.target=System.err

[22/27] hbase git commit: HBASE-18511 Default no regions on master

Posted by bu...@apache.org.
HBASE-18511 Default no regions on master


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

Branch: refs/heads/HBASE-18467
Commit: a8e0267c00409da7e3e03cfb34168787873ef075
Parents: 4376233
Author: Michael Stack <st...@apache.org>
Authored: Sat Aug 5 15:28:40 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Tue Aug 8 12:11:02 2017 +0800

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java | 20 ++++++++------------
 1 file changed, 8 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a8e0267c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 8f5b6f5..445f80f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -1111,12 +1111,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   private static final Random RANDOM = new Random(System.currentTimeMillis());
   private static final Log LOG = LogFactory.getLog(BaseLoadBalancer.class);
 
-  // Regions of these tables are put on the master by default.
-  private static final String[] DEFAULT_TABLES_ON_MASTER =
-    new String[] {AccessControlLists.ACL_TABLE_NAME.getNameAsString(),
-      TableName.NAMESPACE_TABLE_NAME.getNameAsString(),
-      TableName.META_TABLE_NAME.getNameAsString()};
-
+  // Master carries no regions by default.
+  private static final String[] DEFAULT_TABLES_ON_MASTER = new String[] {};
   public static final String TABLES_ON_MASTER =
     "hbase.balancer.tablesOnMaster";
 
@@ -1127,18 +1123,18 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
   protected MasterServices services;
 
   /**
-   * By default, regions of some small system tables such as meta,
-   * namespace, and acl are assigned to the active master. If you don't
-   * want to assign any region to the active master, you need to
-   * configure "hbase.balancer.tablesOnMaster" to "none".
+   * By default, master carries no regions. This method returns null.
+   * If you want master to carry system tables say, then set
+   * TABLES_ON_MASTER to AccessControlLists.ACL_TABLE_NAME.getNameAsString(),
+   * TableName.NAMESPACE_TABLE_NAME.getNameAsString(),
+   * TableName.META_TABLE_NAME.getNameAsString()
    */
   protected static String[] getTablesOnMaster(Configuration conf) {
     String valueString = conf.get(TABLES_ON_MASTER);
     if (valueString == null) {
       return DEFAULT_TABLES_ON_MASTER;
     }
-    valueString = valueString.trim();
-    if (valueString.equalsIgnoreCase("none")) {
+    if (valueString == null || valueString.trim().isEmpty()) {
       return null;
     }
     return StringUtils.getStrings(valueString);

[20/27] hbase git commit: HBASE-18527 Update nightly job to account for changes in jenkins plugins.

Posted by bu...@apache.org.
HBASE-18527 Update nightly job to account for changes in jenkins plugins.

 - Stop presuming job will be configured with checkout into a 'component' dir
 - Disable autocheckout
 - Specify checkout within 'component' dir
 - leave the workspace around on pipeline failure so we can browse it.


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

Branch: refs/heads/HBASE-18467
Commit: 7e7461e42a8cbdb470d531276ad59df46e1fd4e1
Parents: fd76eb3
Author: Sean Busbey <bu...@apache.org>
Authored: Sun Aug 6 14:46:56 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Mon Aug 7 11:24:00 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 13 ++++++++-----
 1 file changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7e7461e4/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 94d8c3f..1f01a47 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -27,6 +27,7 @@ pipeline {
     buildDiscarder(logRotator(numToKeepStr: '30'))
     timeout (time: 6, unit: 'HOURS')
     timestamps()
+    skipDefaultCheckout()
   }
   environment {
     TOOLS = "${env.WORKSPACE}/tools"
@@ -59,6 +60,13 @@ pipeline {
     booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
   }
   stages {
+    stage ('scm checkout') {
+      steps {
+        dir('component') {
+          checkout scm
+        }
+      }
+    }
     stage ('yetus install') {
       steps {
         sh  '''#!/usr/bin/env bash
@@ -298,9 +306,4 @@ END
       }
     }
   }
-  post {
-    failure {
-      deleteDir()
-    }
-  }
 }


[02/27] hbase git commit: HBASE-18470 Remove the redundant comma from RetriesExhaustedWithDetailsException#getDesc

Posted by bu...@apache.org.
HBASE-18470 Remove the redundant comma from RetriesExhaustedWithDetailsException#getDesc

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>


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

Branch: refs/heads/HBASE-18467
Commit: fe890b70ace30f35cce947de26a64fb646290219
Parents: 855dd48
Author: Benedict Jin <15...@qq.com>
Authored: Thu Aug 3 14:55:23 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Aug 3 15:42:53 2017 +0800

----------------------------------------------------------------------
 .../hbase/client/RetriesExhaustedWithDetailsException.java       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fe890b70/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
index 70d5548..e1a6754 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
@@ -111,10 +111,10 @@ extends RetriesExhaustedException {
     Set<String> uniqAddr = new HashSet<>();
     uniqAddr.addAll(hostnamePort);
 
-    for(String addr : uniqAddr) {
+    for (String addr : uniqAddr) {
       addrs.append(addr).append(", ");
     }
-    return addrs.toString();
+    return uniqAddr.isEmpty() ? addrs.toString() : addrs.substring(0, addrs.length() - 2);
   }
 
   public String getExhaustiveDescription() {