You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2015/12/15 18:43:15 UTC

[01/26] hbase git commit: HBASE-14954 IllegalArgumentException was thrown when doing online configuration change in CompactSplitThread (Victor Xu)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 7bfbb6a3c -> 64c55b489


HBASE-14954 IllegalArgumentException was thrown when doing online configuration change in CompactSplitThread (Victor Xu)


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

Branch: refs/heads/hbase-12439
Commit: 0e147a9d6e53e71ad2e57f512b4d3e1eeeac0b78
Parents: 7bfbb6a
Author: tedyu <yu...@gmail.com>
Authored: Wed Dec 9 07:18:08 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Dec 9 07:18:08 2015 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/CompactSplitThread.java  |  46 ++++++--
 .../regionserver/TestCompactSplitThread.java    | 104 +++++++++++++++++++
 2 files changed, 141 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0e147a9d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index 6ce90bc..f54f008 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -618,8 +618,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       LOG.info("Changing the value of " + LARGE_COMPACTION_THREADS +
               " from " + this.longCompactions.getCorePoolSize() + " to " +
               largeThreads);
-      this.longCompactions.setMaximumPoolSize(largeThreads);
-      this.longCompactions.setCorePoolSize(largeThreads);
+      if(this.longCompactions.getCorePoolSize() < largeThreads) {
+        this.longCompactions.setMaximumPoolSize(largeThreads);
+        this.longCompactions.setCorePoolSize(largeThreads);
+      } else {
+        this.longCompactions.setCorePoolSize(largeThreads);
+        this.longCompactions.setMaximumPoolSize(largeThreads);
+      }
     }
 
     int smallThreads = newConf.getInt(SMALL_COMPACTION_THREADS,
@@ -628,8 +633,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       LOG.info("Changing the value of " + SMALL_COMPACTION_THREADS +
                 " from " + this.shortCompactions.getCorePoolSize() + " to " +
                 smallThreads);
-      this.shortCompactions.setMaximumPoolSize(smallThreads);
-      this.shortCompactions.setCorePoolSize(smallThreads);
+      if(this.shortCompactions.getCorePoolSize() < smallThreads) {
+        this.shortCompactions.setMaximumPoolSize(smallThreads);
+        this.shortCompactions.setCorePoolSize(smallThreads);
+      } else {
+        this.shortCompactions.setCorePoolSize(smallThreads);
+        this.shortCompactions.setMaximumPoolSize(smallThreads);
+      }
     }
 
     int splitThreads = newConf.getInt(SPLIT_THREADS,
@@ -638,8 +648,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       LOG.info("Changing the value of " + SPLIT_THREADS +
                 " from " + this.splits.getCorePoolSize() + " to " +
                 splitThreads);
-      this.splits.setMaximumPoolSize(smallThreads);
-      this.splits.setCorePoolSize(smallThreads);
+      if(this.splits.getCorePoolSize() < splitThreads) {
+        this.splits.setMaximumPoolSize(splitThreads);
+        this.splits.setCorePoolSize(splitThreads);
+      } else {
+        this.splits.setCorePoolSize(splitThreads);
+        this.splits.setMaximumPoolSize(splitThreads);
+      }
     }
 
     int mergeThreads = newConf.getInt(MERGE_THREADS,
@@ -648,8 +663,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       LOG.info("Changing the value of " + MERGE_THREADS +
                 " from " + this.mergePool.getCorePoolSize() + " to " +
                 mergeThreads);
-      this.mergePool.setMaximumPoolSize(smallThreads);
-      this.mergePool.setCorePoolSize(smallThreads);
+      if(this.mergePool.getCorePoolSize() < mergeThreads) {
+        this.mergePool.setMaximumPoolSize(mergeThreads);
+        this.mergePool.setCorePoolSize(mergeThreads);
+      } else {
+        this.mergePool.setCorePoolSize(mergeThreads);
+        this.mergePool.setMaximumPoolSize(mergeThreads);
+      }
     }
 
     CompactionThroughputController old = this.compactionThroughputController;
@@ -668,10 +688,18 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
     return this.shortCompactions.getCorePoolSize();
   }
 
-  public int getLargeCompactionThreadNum() {
+  protected int getLargeCompactionThreadNum() {
     return this.longCompactions.getCorePoolSize();
   }
 
+  protected int getSplitThreadNum() {
+    return this.splits.getCorePoolSize();
+  }
+
+  protected int getMergeThreadNum() {
+    return this.mergePool.getCorePoolSize();
+  }
+
   /**
    * {@inheritDoc}
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/0e147a9d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
new file mode 100644
index 0000000..022279a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
@@ -0,0 +1,104 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
+import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category(MediumTests.class)
+public class TestCompactSplitThread {
+  private static final Log LOG = LogFactory.getLog(TestCompactSplitThread.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final TableName tableName = TableName.valueOf(getClass().getSimpleName());
+  private final byte[] family = Bytes.toBytes("f");
+
+  @Test
+  public void testThreadPoolSizeTuning() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 3);
+    conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 4);
+    conf.setInt(CompactSplitThread.SPLIT_THREADS, 5);
+    conf.setInt(CompactSplitThread.MERGE_THREADS, 6);
+    TEST_UTIL.startMiniCluster(1);
+    Connection conn = ConnectionFactory.createConnection(conf);
+    try {
+      HTableDescriptor htd = new HTableDescriptor(tableName);
+      htd.addFamily(new HColumnDescriptor(family));
+      htd.setCompactionEnabled(false);
+      TEST_UTIL.getHBaseAdmin().createTable(htd);
+      TEST_UTIL.waitTableAvailable(tableName);
+      HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
+
+      // check initial configuration of thread pool sizes
+      assertEquals(3, regionServer.compactSplitThread.getLargeCompactionThreadNum());
+      assertEquals(4, regionServer.compactSplitThread.getSmallCompactionThreadNum());
+      assertEquals(5, regionServer.compactSplitThread.getSplitThreadNum());
+      assertEquals(6, regionServer.compactSplitThread.getMergeThreadNum());
+
+      // change bigger configurations and do online update
+      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 4);
+      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 5);
+      conf.setInt(CompactSplitThread.SPLIT_THREADS, 6);
+      conf.setInt(CompactSplitThread.MERGE_THREADS, 7);
+      try {
+        regionServer.compactSplitThread.onConfigurationChange(conf);
+      } catch (IllegalArgumentException iae) {
+        Assert.fail("Update bigger configuration failed!");
+      }
+
+      // check again after online update
+      assertEquals(4, regionServer.compactSplitThread.getLargeCompactionThreadNum());
+      assertEquals(5, regionServer.compactSplitThread.getSmallCompactionThreadNum());
+      assertEquals(6, regionServer.compactSplitThread.getSplitThreadNum());
+      assertEquals(7, regionServer.compactSplitThread.getMergeThreadNum());
+
+      // change smaller configurations and do online update
+      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 2);
+      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 3);
+      conf.setInt(CompactSplitThread.SPLIT_THREADS, 4);
+      conf.setInt(CompactSplitThread.MERGE_THREADS, 5);
+      try {
+        regionServer.compactSplitThread.onConfigurationChange(conf);
+      } catch (IllegalArgumentException iae) {
+        Assert.fail("Update smaller configuration failed!");
+      }
+
+      // check again after online update
+      assertEquals(2, regionServer.compactSplitThread.getLargeCompactionThreadNum());
+      assertEquals(3, regionServer.compactSplitThread.getSmallCompactionThreadNum());
+      assertEquals(4, regionServer.compactSplitThread.getSplitThreadNum());
+      assertEquals(5, regionServer.compactSplitThread.getMergeThreadNum());
+    } finally {
+      conn.close();
+      TEST_UTIL.shutdownMiniCluster();
+    }
+  }
+}


[18/26] hbase git commit: HBASE-14953 Replication: retry on RejectedExecutionException

Posted by sy...@apache.org.
HBASE-14953 Replication: retry on RejectedExecutionException

In HBaseInterClusterReplicationEndpoint, we fail the whole batch
in case of a RejectedExecutionException on an individual sub-batch.
We should let the submitted sub-batches finish and retry only for
the remaining ones.

Signed-off-by: Elliott Clark <ec...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 453a66c3b9d4f51311db6acc5cba84290dbd29ed
Parents: 22b95ae
Author: Ashu Pachauri <as...@gmail.com>
Authored: Tue Dec 8 14:25:41 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Dec 11 13:15:22 2015 -0800

----------------------------------------------------------------------
 .../regionserver/HBaseInterClusterReplicationEndpoint.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/453a66c3/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index d51d512..70cc420 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -29,7 +29,7 @@ import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
-import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
@@ -113,8 +113,9 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     // per sink thread pool
     this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
       HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
-    this.exec = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS,
-        new SynchronousQueue<Runnable>());
+    this.exec = new ThreadPoolExecutor(maxThreads, maxThreads, 60, TimeUnit.SECONDS,
+        new LinkedBlockingQueue<Runnable>());
+    this.exec.allowCoreThreadTimeOut(true);
 
     this.replicationBulkLoadDataEnabled =
         conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,


[17/26] hbase git commit: HBASE-14946 Don't allow multi's to over run the max result size.

Posted by sy...@apache.org.
HBASE-14946 Don't allow multi's to over run the max result size.


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

Branch: refs/heads/hbase-12439
Commit: 22b95aebcd7fc742412ab514520008fda5e327de
Parents: cff664c
Author: stack <st...@apache.org>
Authored: Thu Dec 10 21:25:21 2015 -0800
Committer: stack <st...@apache.org>
Committed: Thu Dec 10 21:25:21 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/MultiActionResultTooLarge.java     | 6 +++++-
 .../org/apache/hadoop/hbase/RetryImmediatelyException.java     | 4 ++++
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/22b95aeb/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
index d06eea1..fdff554 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
@@ -18,13 +18,17 @@
 
 package org.apache.hadoop.hbase;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
 /**
  * Exception thrown when the result needs to be chunked on the server side.
  * It signals that retries should happen right away and not count against the number of
  * retries because some of the multi was a success.
  */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public class MultiActionResultTooLarge extends RetryImmediatelyException {
-
   public MultiActionResultTooLarge(String s) {
     super(s);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/22b95aeb/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
index 1b39904..e0b90fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
@@ -19,7 +19,11 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
 public class RetryImmediatelyException extends IOException {
   public RetryImmediatelyException(String s) {
     super(s);


[09/26] hbase git commit: HBASE-14901 Remove duplicate code to create/manage encryption keys

Posted by sy...@apache.org.
HBASE-14901 Remove duplicate code to create/manage encryption keys

Signed-off-by: Gary Helmling <ga...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: 9511150bd60e5149856c23c90422e2da7114892e
Parents: 6f8d5e8
Author: Nate Edel <na...@gmail.com>
Authored: Fri Dec 4 12:30:05 2015 -0800
Committer: Gary Helmling <ga...@apache.org>
Committed: Thu Dec 10 14:38:43 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/security/EncryptionUtil.java   | 98 +++++++++++++++++++-
 .../hbase/security/TestEncryptionUtil.java      |  6 ++
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  | 34 +------
 .../org/apache/hadoop/hbase/mob/MobUtils.java   | 82 ++--------------
 .../compactions/PartitionedMobCompactor.java    |  3 +-
 .../hbase/mob/mapreduce/MemStoreWrapper.java    |  3 +-
 .../hadoop/hbase/regionserver/HStore.java       | 62 +------------
 7 files changed, 122 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
index c787efe..d89d96c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/EncryptionUtil.java
@@ -26,7 +26,10 @@ import java.security.SecureRandom;
 
 import javax.crypto.spec.SecretKeySpec;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -41,11 +44,18 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class EncryptionUtil {
+public final class EncryptionUtil {
+  static private final Log LOG = LogFactory.getLog(EncryptionUtil.class);
 
   static private final SecureRandom RNG = new SecureRandom();
 
   /**
+   * Private constructor to keep this class from being instantiated.
+   */
+  private EncryptionUtil() {
+  }
+
+  /**
    * Protect a key by encrypting it with the secret key of the given subject.
    * The configuration must be set up correctly for key alias resolution.
    * @param conf configuration
@@ -159,4 +169,90 @@ public class EncryptionUtil {
     return getUnwrapKey(conf, subject, wrappedKey, cipher);
   }
 
+  /**
+   * Helper to create an encyption context.
+   *
+   * @param conf The current configuration.
+   * @param family The current column descriptor.
+   * @return The created encryption context.
+   * @throws IOException if an encryption key for the column cannot be unwrapped
+   */
+  public static Encryption.Context createEncryptionContext(Configuration conf,
+    HColumnDescriptor family) throws IOException {
+    Encryption.Context cryptoContext = Encryption.Context.NONE;
+    String cipherName = family.getEncryptionType();
+    if (cipherName != null) {
+      Cipher cipher;
+      Key key;
+      byte[] keyBytes = family.getEncryptionKey();
+      if (keyBytes != null) {
+        // Family provides specific key material
+        key = unwrapKey(conf, keyBytes);
+        // Use the algorithm the key wants
+        cipher = Encryption.getCipher(conf, key.getAlgorithm());
+        if (cipher == null) {
+          throw new RuntimeException("Cipher '" + key.getAlgorithm() + "' is not available");
+        }
+        // Fail if misconfigured
+        // We use the encryption type specified in the column schema as a sanity check on
+        // what the wrapped key is telling us
+        if (!cipher.getName().equalsIgnoreCase(cipherName)) {
+          throw new RuntimeException("Encryption for family '" + family.getNameAsString()
+            + "' configured with type '" + cipherName + "' but key specifies algorithm '"
+            + cipher.getName() + "'");
+        }
+      } else {
+        // Family does not provide key material, create a random key
+        cipher = Encryption.getCipher(conf, cipherName);
+        if (cipher == null) {
+          throw new RuntimeException("Cipher '" + cipherName + "' is not available");
+        }
+        key = cipher.getRandomKey();
+      }
+      cryptoContext = Encryption.newContext(conf);
+      cryptoContext.setCipher(cipher);
+      cryptoContext.setKey(key);
+    }
+    return cryptoContext;
+  }
+
+  /**
+   * Helper for {@link #unwrapKey(Configuration, String, byte[])} which automatically uses the
+   * configured master and alternative keys, rather than having to specify a key type to unwrap
+   * with.
+   *
+   * The configuration must be set up correctly for key alias resolution.
+   *
+   * @param conf the current configuration
+   * @param keyBytes the key encrypted by master (or alternative) to unwrap
+   * @return the key bytes, decrypted
+   * @throws IOException if the key cannot be unwrapped
+   */
+  public static Key unwrapKey(Configuration conf, byte[] keyBytes) throws IOException {
+    Key key;
+    String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
+      User.getCurrent().getShortName());
+    try {
+      // First try the master key
+      key = unwrapKey(conf, masterKeyName, keyBytes);
+    } catch (KeyException e) {
+      // If the current master key fails to unwrap, try the alternate, if
+      // one is configured
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
+      }
+      String alternateKeyName =
+        conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
+      if (alternateKeyName != null) {
+        try {
+          key = unwrapKey(conf, alternateKeyName, keyBytes);
+        } catch (KeyException ex) {
+          throw new IOException(ex);
+        }
+      } else {
+        throw new IOException(e);
+      }
+    }
+    return key;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java
index b0e3464..edcee70 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/security/TestEncryptionUtil.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.security.Key;
 import java.security.KeyException;
 import java.security.SecureRandom;
@@ -28,7 +29,9 @@ import java.security.SecureRandom;
 import javax.crypto.spec.SecretKeySpec;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.io.crypto.aes.AES;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -39,6 +42,9 @@ import org.junit.experimental.categories.Category;
 
 @Category({ClientTests.class, SmallTests.class})
 public class TestEncryptionUtil {
+  // There does not seem to be a ready way to test either getKeyFromBytesOrMasterKey
+  // or createEncryptionContext, and the existing code under MobUtils appeared to be
+  // untested.  Not ideal!
 
   @Test
   public void testKeyWrapping() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 930f42a..4e2ca7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -21,26 +21,25 @@ import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.Key;
-import java.security.KeyException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.OffheapKeyValue;
 import org.apache.hadoop.hbase.ShareableMemory;
 import org.apache.hadoop.hbase.SizeCachedKeyValue;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.compress.Compression;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
@@ -1817,29 +1815,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     if (keyBytes != null) {
       Encryption.Context cryptoContext = Encryption.newContext(conf);
       Key key;
-      String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
-        User.getCurrent().getShortName());
-      try {
-        // First try the master key
-        key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
-      } catch (KeyException e) {
-        // If the current master key fails to unwrap, try the alternate, if
-        // one is configured
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
-        }
-        String alternateKeyName =
-          conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
-        if (alternateKeyName != null) {
-          try {
-            key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
-          } catch (KeyException ex) {
-            throw new IOException(ex);
-          }
-        } else {
-          throw new IOException(e);
-        }
-      }
+      key = EncryptionUtil.unwrapKey(conf, keyBytes);
       // Use the algorithm the key wants
       Cipher cipher = Encryption.getCipher(conf, key.getAlgorithm());
       if (cipher == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 424a39b..d654788 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.mob;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.security.Key;
-import java.security.KeyException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -58,7 +56,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.crypto.Cipher;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -70,8 +67,6 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.security.EncryptionUtil;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -82,7 +77,7 @@ import org.apache.hadoop.hbase.util.Threads;
  * The mob utilities
  */
 @InterfaceAudience.Private
-public class MobUtils {
+public final class MobUtils {
 
   private static final Log LOG = LogFactory.getLog(MobUtils.class);
 
@@ -94,6 +89,13 @@ public class MobUtils {
     }
   };
 
+
+  /**
+   * Private constructor to keep this class from being instantiated.
+   */
+  private MobUtils() {
+  }
+
   /**
    * Formats a date to a string.
    * @param date The date.
@@ -775,74 +777,6 @@ public class MobUtils {
   }
 
   /**
-   * Creates the encyption context.
-   * @param conf The current configuration.
-   * @param family The current column descriptor.
-   * @return The encryption context.
-   * @throws IOException
-   */
-  public static Encryption.Context createEncryptionContext(Configuration conf,
-    HColumnDescriptor family) throws IOException {
-    // TODO the code is repeated, and needs to be unified.
-    Encryption.Context cryptoContext = Encryption.Context.NONE;
-    String cipherName = family.getEncryptionType();
-    if (cipherName != null) {
-      Cipher cipher;
-      Key key;
-      byte[] keyBytes = family.getEncryptionKey();
-      if (keyBytes != null) {
-        // Family provides specific key material
-        String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User
-          .getCurrent().getShortName());
-        try {
-          // First try the master key
-          key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
-        } catch (KeyException e) {
-          // If the current master key fails to unwrap, try the alternate, if
-          // one is configured
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
-          }
-          String alternateKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
-          if (alternateKeyName != null) {
-            try {
-              key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
-            } catch (KeyException ex) {
-              throw new IOException(ex);
-            }
-          } else {
-            throw new IOException(e);
-          }
-        }
-        // Use the algorithm the key wants
-        cipher = Encryption.getCipher(conf, key.getAlgorithm());
-        if (cipher == null) {
-          throw new RuntimeException("Cipher '" + key.getAlgorithm() + "' is not available");
-        }
-        // Fail if misconfigured
-        // We use the encryption type specified in the column schema as a sanity check on
-        // what the wrapped key is telling us
-        if (!cipher.getName().equalsIgnoreCase(cipherName)) {
-          throw new RuntimeException("Encryption for family '" + family.getNameAsString()
-            + "' configured with type '" + cipherName + "' but key specifies algorithm '"
-            + cipher.getName() + "'");
-        }
-      } else {
-        // Family does not provide key material, create a random key
-        cipher = Encryption.getCipher(conf, cipherName);
-        if (cipher == null) {
-          throw new RuntimeException("Cipher '" + cipherName + "' is not available");
-        }
-        key = cipher.getRandomKey();
-      }
-      cryptoContext = Encryption.newContext(conf);
-      cryptoContext.setCipher(cipher);
-      cryptoContext.setKey(key);
-    }
-    return cryptoContext;
-  }
-
-  /**
    * Checks whether this table has mob-enabled columns.
    * @param htd The current table descriptor.
    * @return Whether this table has mob-enabled columns.

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index dabedfd..ab9ee7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 
@@ -113,7 +114,7 @@ public class PartitionedMobCompactor extends MobCompactor {
     copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
     compactionCacheConfig = new CacheConfig(copyOfConf);
     tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
-    cryptoContext = MobUtils.createEncryptionContext(copyOfConf, column);
+    cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
index 08e6753..3daef7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/MemStoreWrapper.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.MemStore;
 import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.Reducer.Context;
 
@@ -90,7 +91,7 @@ public class MemStoreWrapper {
     flushSize = this.conf.getLong(MobConstants.MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE,
         MobConstants.DEFAULT_MOB_SWEEP_TOOL_COMPACTION_MEMSTORE_FLUSH_SIZE);
     mobFamilyDir = MobUtils.getMobFamilyPath(conf, table.getName(), hcd.getNameAsString());
-    cryptoContext = MobUtils.createEncryptionContext(conf, hcd);
+    cryptoContext = EncryptionUtil.createEncryptionContext(conf, hcd);
   }
 
   public void setPartitionId(CompactionPartitionId partitionId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9511150b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 50b3de7..49b6c50 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.net.InetSocketAddress;
-import java.security.Key;
-import java.security.KeyException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -65,7 +63,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.crypto.Cipher;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -82,9 +79,9 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
@@ -280,62 +277,7 @@ public class HStore implements Store {
       conf.getInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 10));
     completionService =
         new ExecutorCompletionService<StoreFile>(compactionCleanerthreadPoolExecutor);
-    // Crypto context for new store files
-    String cipherName = family.getEncryptionType();
-    if (cipherName != null) {
-      Cipher cipher;
-      Key key;
-      byte[] keyBytes = family.getEncryptionKey();
-      if (keyBytes != null) {
-        // Family provides specific key material
-        String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
-          User.getCurrent().getShortName());
-        try {
-          // First try the master key
-          key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
-        } catch (KeyException e) {
-          // If the current master key fails to unwrap, try the alternate, if
-          // one is configured
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
-          }
-          String alternateKeyName =
-            conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
-          if (alternateKeyName != null) {
-            try {
-              key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
-            } catch (KeyException ex) {
-              throw new IOException(ex);
-            }
-          } else {
-            throw new IOException(e);
-          }
-        }
-        // Use the algorithm the key wants
-        cipher = Encryption.getCipher(conf, key.getAlgorithm());
-        if (cipher == null) {
-          throw new RuntimeException("Cipher '" + key.getAlgorithm() + "' is not available");
-        }
-        // Fail if misconfigured
-        // We use the encryption type specified in the column schema as a sanity check on
-        // what the wrapped key is telling us
-        if (!cipher.getName().equalsIgnoreCase(cipherName)) {
-          throw new RuntimeException("Encryption for family '" + family.getNameAsString() +
-            "' configured with type '" + cipherName +
-            "' but key specifies algorithm '" + cipher.getName() + "'");
-        }
-      } else {
-        // Family does not provide key material, create a random key
-        cipher = Encryption.getCipher(conf, cipherName);
-        if (cipher == null) {
-          throw new RuntimeException("Cipher '" + cipherName + "' is not available");
-        }
-        key = cipher.getRandomKey();
-      }
-      cryptoContext = Encryption.newContext(conf);
-      cryptoContext.setCipher(cipher);
-      cryptoContext.setKey(key);
-    }
+    cryptoContext = EncryptionUtil.createEncryptionContext(conf, family);
   }
 
   /**


[06/26] hbase git commit: HBASE-13153 Bulk Loaded HFile Replication (Ashish Singhi)

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
index 457d859..db98083 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
@@ -85,17 +85,16 @@ public interface WALActionsListener {
   );
 
   /**
-   *
    * @param htd
    * @param logKey
-   * @param logEdit
-   * TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)}
-   * It only exists to get scope when replicating.  Scope should be in the WALKey and not need
-   * us passing in a <code>htd</code>.
+   * @param logEdit TODO: Retire this in favor of
+   *          {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} It only exists to get
+   *          scope when replicating. Scope should be in the WALKey and not need us passing in a
+   *          <code>htd</code>.
+   * @throws IOException If failed to parse the WALEdit
    */
-  void visitLogEntryBeforeWrite(
-    HTableDescriptor htd, WALKey logKey, WALEdit logEdit
-  );
+  void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
+      throws IOException;
 
   /**
    * For notification post append to the writer.  Used by metrics system at least.
@@ -136,7 +135,9 @@ public interface WALActionsListener {
     public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey, WALEdit logEdit) {}
 
     @Override
-    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {}
+    public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
+        throws IOException {
+    }
 
     @Override
     public void postAppend(final long entryLen, final long elapsedTimeMillis) {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
index 3501f3e..f97ec15 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
@@ -18,13 +18,21 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 import java.util.NavigableMap;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 /**
@@ -32,6 +40,7 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 @InterfaceAudience.Private
 public class ScopeWALEntryFilter implements WALEntryFilter {
+  private static final Log LOG = LogFactory.getLog(ScopeWALEntryFilter.class);
 
   @Override
   public Entry filter(Entry entry) {
@@ -41,13 +50,27 @@ public class ScopeWALEntryFilter implements WALEntryFilter {
     }
     ArrayList<Cell> cells = entry.getEdit().getCells();
     int size = cells.size();
+    byte[] fam;
     for (int i = size - 1; i >= 0; i--) {
       Cell cell = cells.get(i);
-      // The scope will be null or empty if
-      // there's nothing to replicate in that WALEdit
-      byte[] fam = CellUtil.cloneFamily(cell);
-      if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
-        cells.remove(i);
+      // If a bulk load entry has a scope then that means user has enabled replication for bulk load
+      // hfiles.
+      // TODO There is a similar logic in TableCfWALEntryFilter but data structures are different so
+      // cannot refactor into one now, can revisit and see if any way to unify them.
+      if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
+        Cell filteredBulkLoadEntryCell = filterBulkLoadEntries(scopes, cell);
+        if (filteredBulkLoadEntryCell != null) {
+          cells.set(i, filteredBulkLoadEntryCell);
+        } else {
+          cells.remove(i);
+        }
+      } else {
+        // The scope will be null or empty if
+        // there's nothing to replicate in that WALEdit
+        fam = CellUtil.cloneFamily(cell);
+        if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
+          cells.remove(i);
+        }
       }
     }
     if (cells.size() < size / 2) {
@@ -56,4 +79,41 @@ public class ScopeWALEntryFilter implements WALEntryFilter {
     return entry;
   }
 
+  private Cell filterBulkLoadEntries(NavigableMap<byte[], Integer> scopes, Cell cell) {
+    byte[] fam;
+    BulkLoadDescriptor bld = null;
+    try {
+      bld = WALEdit.getBulkLoadDescriptor(cell);
+    } catch (IOException e) {
+      LOG.warn("Failed to get bulk load events information from the WAL file.", e);
+      return cell;
+    }
+    List<StoreDescriptor> storesList = bld.getStoresList();
+    // Copy the StoreDescriptor list and update it as storesList is a unmodifiableList
+    List<StoreDescriptor> copiedStoresList = new ArrayList<StoreDescriptor>(storesList);
+    Iterator<StoreDescriptor> copiedStoresListIterator = copiedStoresList.iterator();
+    boolean anyStoreRemoved = false;
+    while (copiedStoresListIterator.hasNext()) {
+      StoreDescriptor sd = copiedStoresListIterator.next();
+      fam = sd.getFamilyName().toByteArray();
+      if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
+        copiedStoresListIterator.remove();
+        anyStoreRemoved = true;
+      }
+    }
+
+    if (!anyStoreRemoved) {
+      return cell;
+    } else if (copiedStoresList.isEmpty()) {
+      return null;
+    }
+    BulkLoadDescriptor.Builder newDesc =
+        BulkLoadDescriptor.newBuilder().setTableName(bld.getTableName())
+            .setEncodedRegionName(bld.getEncodedRegionName())
+            .setBulkloadSeqNum(bld.getBulkloadSeqNum());
+    newDesc.addAllStores(copiedStoresList);
+    BulkLoadDescriptor newBulkLoadDescriptor = newDesc.build();
+    return CellUtil.createCell(CellUtil.cloneRow(cell), WALEdit.METAFAMILY, WALEdit.BULK_LOAD,
+      cell.getTimestamp(), cell.getTypeByte(), newBulkLoadDescriptor.toByteArray());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java
index 642ee8a..f10849b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java
@@ -18,14 +18,20 @@
 
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
@@ -52,19 +58,36 @@ public class TableCfWALEntryFilter implements WALEntryFilter {
     }
     int size = cells.size();
 
+    // If null means user has explicitly not configured any table CFs so all the tables data are
+    // applicable for replication
+    if (tableCFs == null) {
+      return entry;
+    }
     // return null(prevent replicating) if logKey's table isn't in this peer's
-    // replicable table list (empty tableCFs means all table are replicable)
-    if (tableCFs != null && !tableCFs.containsKey(tabName)) {
+    // replicable table list
+    if (!tableCFs.containsKey(tabName)) {
       return null;
     } else {
-      List<String> cfs = (tableCFs == null) ? null : tableCFs.get(tabName);
+      List<String> cfs = tableCFs.get(tabName);
       for (int i = size - 1; i >= 0; i--) {
         Cell cell = cells.get(i);
-        // ignore(remove) kv if its cf isn't in the replicable cf list
-        // (empty cfs means all cfs of this table are replicable)
-        if ((cfs != null) && !cfs.contains(Bytes.toString(
-            cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()))) {
-          cells.remove(i);
+        // TODO There is a similar logic in ScopeWALEntryFilter but data structures are different so
+        // cannot refactor into one now, can revisit and see if any way to unify them.
+        // Filter bulk load entries separately
+        if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
+          Cell filteredBulkLoadEntryCell = filterBulkLoadEntries(cfs, cell);
+          if (filteredBulkLoadEntryCell != null) {
+            cells.set(i, filteredBulkLoadEntryCell);
+          } else {
+            cells.remove(i);
+          }
+        } else {
+          // ignore(remove) kv if its cf isn't in the replicable cf list
+          // (empty cfs means all cfs of this table are replicable)
+          if ((cfs != null) && !cfs.contains(Bytes.toString(cell.getFamilyArray(),
+            cell.getFamilyOffset(), cell.getFamilyLength()))) {
+            cells.remove(i);
+          }
         }
       }
     }
@@ -74,4 +97,41 @@ public class TableCfWALEntryFilter implements WALEntryFilter {
     return entry;
   }
 
+  private Cell filterBulkLoadEntries(List<String> cfs, Cell cell) {
+    byte[] fam;
+    BulkLoadDescriptor bld = null;
+    try {
+      bld = WALEdit.getBulkLoadDescriptor(cell);
+    } catch (IOException e) {
+      LOG.warn("Failed to get bulk load events information from the WAL file.", e);
+      return cell;
+    }
+    List<StoreDescriptor> storesList = bld.getStoresList();
+    // Copy the StoreDescriptor list and update it as storesList is a unmodifiableList
+    List<StoreDescriptor> copiedStoresList = new ArrayList<StoreDescriptor>(storesList);
+    Iterator<StoreDescriptor> copiedStoresListIterator = copiedStoresList.iterator();
+    boolean anyStoreRemoved = false;
+    while (copiedStoresListIterator.hasNext()) {
+      StoreDescriptor sd = copiedStoresListIterator.next();
+      fam = sd.getFamilyName().toByteArray();
+      if (cfs != null && !cfs.contains(Bytes.toString(fam))) {
+        copiedStoresListIterator.remove();
+        anyStoreRemoved = true;
+      }
+    }
+
+    if (!anyStoreRemoved) {
+      return cell;
+    } else if (copiedStoresList.isEmpty()) {
+      return null;
+    }
+    BulkLoadDescriptor.Builder newDesc =
+        BulkLoadDescriptor.newBuilder().setTableName(bld.getTableName())
+            .setEncodedRegionName(bld.getEncodedRegionName())
+            .setBulkloadSeqNum(bld.getBulkloadSeqNum());
+    newDesc.addAllStores(copiedStoresList);
+    BulkLoadDescriptor newBulkLoadDescriptor = newDesc.build();
+    return CellUtil.createCell(CellUtil.cloneRow(cell), WALEdit.METAFAMILY, WALEdit.BULK_LOAD,
+      cell.getTimestamp(), cell.getTypeByte(), newBulkLoadDescriptor.toByteArray());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
new file mode 100644
index 0000000..9bfea4b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication.master;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
+import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
+import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
+ * deleting it from hfile archive directory.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
+  private static final Log LOG = LogFactory.getLog(ReplicationHFileCleaner.class);
+  private ZooKeeperWatcher zkw;
+  private ReplicationQueuesClient rqc;
+  private boolean stopped = false;
+  private boolean aborted;
+
+  @Override
+  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
+    // all members of this class are null if replication is disabled,
+    // so we cannot filter the files
+    if (this.getConf() == null) {
+      return files;
+    }
+
+    final Set<String> hfileRefs;
+    try {
+      // The concurrently created new hfile entries in ZK may not be included in the return list,
+      // but they won't be deleted because they're not in the checking set.
+      hfileRefs = loadHFileRefsFromPeers();
+    } catch (KeeperException e) {
+      LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files");
+      return Collections.emptyList();
+    }
+    return Iterables.filter(files, new Predicate<FileStatus>() {
+      @Override
+      public boolean apply(FileStatus file) {
+        String hfile = file.getPath().getName();
+        boolean foundHFileRefInQueue = hfileRefs.contains(hfile);
+        if (LOG.isDebugEnabled()) {
+          if (foundHFileRefInQueue) {
+            LOG.debug("Found hfile reference in ZK, keeping: " + hfile);
+          } else {
+            LOG.debug("Did not find hfile reference in ZK, deleting: " + hfile);
+          }
+        }
+        return !foundHFileRefInQueue;
+      }
+    });
+  }
+
+  /**
+   * Load all hfile references in all replication queues from ZK. This method guarantees to return a
+   * snapshot which contains all hfile references in the zookeeper at the start of this call.
+   * However, some newly created hfile references during the call may not be included.
+   */
+  private Set<String> loadHFileRefsFromPeers() throws KeeperException {
+    Set<String> hfileRefs = Sets.newHashSet();
+    List<String> listOfPeers;
+    for (int retry = 0;; retry++) {
+      int v0 = rqc.getHFileRefsNodeChangeVersion();
+      hfileRefs.clear();
+      listOfPeers = rqc.getAllPeersFromHFileRefsQueue();
+      if (listOfPeers == null) {
+        LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
+        return ImmutableSet.of();
+      }
+      for (String id : listOfPeers) {
+        List<String> peerHFileRefs = rqc.getReplicableHFiles(id);
+        if (peerHFileRefs != null) {
+          hfileRefs.addAll(peerHFileRefs);
+        }
+      }
+      int v1 = rqc.getHFileRefsNodeChangeVersion();
+      if (v0 == v1) {
+        return hfileRefs;
+      }
+      LOG.debug(String.format("Replication hfile references node cversion changed from "
+          + "%d to %d, retry = %d", v0, v1, retry));
+    }
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    // If either replication or replication of bulk load hfiles is disabled, keep all members null
+    if (!(config.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
+      HConstants.REPLICATION_ENABLE_DEFAULT) && config.getBoolean(
+      HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT))) {
+      LOG.warn(HConstants.REPLICATION_ENABLE_KEY
+          + " is not enabled so allowing all hfile references to be deleted. Better to remove "
+          + ReplicationHFileCleaner.class + " from " + HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS
+          + " configuration.");
+      return;
+    }
+    // Make my own Configuration. Then I'll have my own connection to zk that
+    // I can close myself when time comes.
+    Configuration conf = new Configuration(config);
+    super.setConf(conf);
+    try {
+      initReplicationQueuesClient(conf);
+    } catch (IOException e) {
+      LOG.error("Error while configuring " + this.getClass().getName(), e);
+    }
+  }
+
+  private void initReplicationQueuesClient(Configuration conf)
+      throws ZooKeeperConnectionException, IOException {
+    this.zkw = new ZooKeeperWatcher(conf, "replicationHFileCleaner", null);
+    this.rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, this);
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    this.stopped = true;
+    if (this.zkw != null) {
+      LOG.info("Stopping " + this.zkw);
+      this.zkw.close();
+    }
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
+    this.aborted = true;
+    stop(why);
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted;
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus fStat) {
+    Set<String> hfileRefsFromQueue;
+    // all members of this class are null if replication is disabled,
+    // so do not stop from deleting the file
+    if (getConf() == null) {
+      return true;
+    }
+
+    try {
+      hfileRefsFromQueue = loadHFileRefsFromPeers();
+    } catch (KeeperException e) {
+      LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable "
+          + "file for " + fStat.getPath());
+      return false;
+    }
+    return !hfileRefsFromQueue.contains(fStat.getPath().getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DefaultSourceFSConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DefaultSourceFSConfigurationProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DefaultSourceFSConfigurationProvider.java
new file mode 100644
index 0000000..8d5c6d4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DefaultSourceFSConfigurationProvider.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * This will load all the xml configuration files for the source cluster replication ID from
+ * user configured replication configuration directory.
+ */
+@InterfaceAudience.Private
+public class DefaultSourceFSConfigurationProvider implements SourceFSConfigurationProvider {
+  private static final Log LOG = LogFactory.getLog(DefaultSourceFSConfigurationProvider.class);
+  // Map containing all the source clusters configurations against their replication cluster id
+  private Map<String, Configuration> sourceClustersConfs = new HashMap<>();
+  private static final String XML = ".xml";
+
+  @Override
+  public Configuration getConf(Configuration sinkConf, String replicationClusterId)
+      throws IOException {
+    if (sourceClustersConfs.get(replicationClusterId) == null) {
+      synchronized (this.sourceClustersConfs) {
+        if (sourceClustersConfs.get(replicationClusterId) == null) {
+          LOG.info("Loading source cluster FS client conf for cluster " + replicationClusterId);
+          // Load only user provided client configurations.
+          Configuration sourceClusterConf = new Configuration(false);
+
+          String replicationConfDir = sinkConf.get(HConstants.REPLICATION_CONF_DIR);
+          if (replicationConfDir == null) {
+            LOG.debug(HConstants.REPLICATION_CONF_DIR + " is not configured.");
+            URL resource = HBaseConfiguration.class.getClassLoader().getResource("hbase-site.xml");
+            if (resource != null) {
+              String path = resource.getPath();
+              replicationConfDir = path.substring(0, path.lastIndexOf("/"));
+            } else {
+              replicationConfDir = System.getenv("HBASE_CONF_DIR");
+            }
+          }
+
+          LOG.info("Loading source cluster " + replicationClusterId
+              + " file system configurations from xml files under directory " + replicationConfDir);
+          File confDir = new File(replicationConfDir, replicationClusterId);
+          String[] listofConfFiles = FileUtil.list(confDir);
+          for (String confFile : listofConfFiles) {
+            if (new File(confDir, confFile).isFile() && confFile.endsWith(XML)) {
+              // Add all the user provided client conf files
+              sourceClusterConf.addResource(new Path(confDir.getPath(), confFile));
+            }
+          }
+          this.sourceClustersConfs.put(replicationClusterId, sourceClusterConf);
+        }
+      }
+    }
+    return this.sourceClustersConfs.get(replicationClusterId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 7c07ecc..d51d512 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -37,24 +37,26 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.ipc.RemoteException;
 
 /**
- * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint} 
+ * A {@link org.apache.hadoop.hbase.replication.ReplicationEndpoint}
  * implementation for replicating to another HBase cluster.
  * For the slave cluster it selects a random number of peers
  * using a replication ratio. For example, if replication ration = 0.1
@@ -84,8 +86,12 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
   // Handles connecting to peer region servers
   private ReplicationSinkManager replicationSinkMgr;
   private boolean peersSelected = false;
+  private String replicationClusterId = "";
   private ThreadPoolExecutor exec;
   private int maxThreads;
+  private Path baseNamespaceDir;
+  private Path hfileArchiveDir;
+  private boolean replicationBulkLoadDataEnabled;
 
   @Override
   public void init(Context context) throws IOException {
@@ -108,7 +114,19 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
       HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
     this.exec = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS,
-      new SynchronousQueue<Runnable>());
+        new SynchronousQueue<Runnable>());
+
+    this.replicationBulkLoadDataEnabled =
+        conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+          HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+    if (this.replicationBulkLoadDataEnabled) {
+      replicationClusterId = this.conf.get(HConstants.REPLICATION_CLUSTER_ID);
+    }
+    // Construct base namespace directory and hfile archive directory path
+    Path rootDir = FSUtils.getRootDir(conf);
+    Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR);
+    baseNamespaceDir = new Path(rootDir, baseNSDir);
+    hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir));
   }
 
   private void decorateConf() {
@@ -317,8 +335,8 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
       try {
         sinkPeer = replicationSinkMgr.getReplicationSink();
         BlockingInterface rrs = sinkPeer.getRegionServer();
-        ReplicationProtbufUtil.replicateWALEntry(rrs,
-            entries.toArray(new Entry[entries.size()]));
+        ReplicationProtbufUtil.replicateWALEntry(rrs, entries.toArray(new Entry[entries.size()]),
+          replicationClusterId, baseNamespaceDir, hfileArchiveDir);
         replicationSinkMgr.reportSinkSuccess(sinkPeer);
         return ordinal;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
new file mode 100644
index 0000000..17f6780
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
@@ -0,0 +1,393 @@
+/*
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.math.BigInteger;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.FsDelegationToken;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
+ * staging directory and then it will use ({@link LoadIncrementalHFiles} to prepare a collection of
+ * {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster.
+ */
+@InterfaceAudience.Private
+public class HFileReplicator {
+  /** Maximum number of threads to allow in pool to copy hfiles during replication */
+  public static final String REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY =
+      "hbase.replication.bulkload.copy.maxthreads";
+  public static final int REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT = 10;
+  /** Number of hfiles to copy per thread during replication */
+  public static final String REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY =
+      "hbase.replication.bulkload.copy.hfiles.perthread";
+  public static final int REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT = 10;
+
+  private static final Log LOG = LogFactory.getLog(HFileReplicator.class);
+  private final String UNDERSCORE = "_";
+  private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
+
+  private Configuration sourceClusterConf;
+  private String sourceBaseNamespaceDirPath;
+  private String sourceHFileArchiveDirPath;
+  private Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap;
+  private FileSystem sinkFs;
+  private FsDelegationToken fsDelegationToken;
+  private UserProvider userProvider;
+  private Configuration conf;
+  private Connection connection;
+  private String hbaseStagingDir;
+  private ThreadPoolExecutor exec;
+  private int maxCopyThreads;
+  private int copiesPerThread;
+
+  public HFileReplicator(Configuration sourceClusterConf,
+      String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
+      Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
+      Connection connection) throws IOException {
+    this.sourceClusterConf = sourceClusterConf;
+    this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
+    this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
+    this.bulkLoadHFileMap = tableQueueMap;
+    this.conf = conf;
+    this.connection = connection;
+
+    userProvider = UserProvider.instantiate(conf);
+    fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
+    this.hbaseStagingDir = conf.get("hbase.bulkload.staging.dir");
+    this.maxCopyThreads =
+        this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
+          REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    builder.setNameFormat("HFileReplicationCallable-%1$d");
+    this.exec =
+        new ThreadPoolExecutor(1, maxCopyThreads, 60, TimeUnit.SECONDS,
+            new LinkedBlockingQueue<Runnable>(), builder.build());
+    this.exec.allowCoreThreadTimeOut(true);
+    this.copiesPerThread =
+        conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
+          REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);
+
+    sinkFs = FileSystem.get(conf);
+  }
+
+  public Void replicate() throws IOException {
+    // Copy all the hfiles to the local file system
+    Map<String, Path> tableStagingDirsMap = copyHFilesToStagingDir();
+
+    int maxRetries = conf.getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
+
+    for (Entry<String, Path> tableStagingDir : tableStagingDirsMap.entrySet()) {
+      String tableNameString = tableStagingDir.getKey();
+      Path stagingDir = tableStagingDir.getValue();
+
+      LoadIncrementalHFiles loadHFiles = null;
+      try {
+        loadHFiles = new LoadIncrementalHFiles(conf);
+      } catch (Exception e) {
+        LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
+            + " data.", e);
+        throw new IOException(e);
+      }
+      Configuration newConf = HBaseConfiguration.create(conf);
+      newConf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
+      loadHFiles.setConf(newConf);
+
+      TableName tableName = TableName.valueOf(tableNameString);
+      Table table = this.connection.getTable(tableName);
+
+      // Prepare collection of queue of hfiles to be loaded(replicated)
+      Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
+      loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
+
+      if (queue.isEmpty()) {
+        LOG.warn("Replication process did not find any files to replicate in directory "
+            + stagingDir.toUri());
+        return null;
+      }
+
+      try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+
+        fsDelegationToken.acquireDelegationToken(sinkFs);
+
+        // Set the staging directory which will be used by LoadIncrementalHFiles for loading the
+        // data
+        loadHFiles.setBulkToken(stagingDir.toString());
+
+        doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
+      } finally {
+        cleanup(stagingDir.toString(), table);
+      }
+    }
+    return null;
+  }
+
+  private void doBulkLoad(LoadIncrementalHFiles loadHFiles, Table table,
+      Deque<LoadQueueItem> queue, RegionLocator locator, int maxRetries) throws IOException {
+    int count = 0;
+    Pair<byte[][], byte[][]> startEndKeys;
+    while (!queue.isEmpty()) {
+      // need to reload split keys each iteration.
+      startEndKeys = locator.getStartEndKeys();
+      if (count != 0) {
+        LOG.warn("Error occured while replicating HFiles, retry attempt " + count + " with "
+            + queue.size() + " files still remaining to replicate.");
+      }
+
+      if (maxRetries != 0 && count >= maxRetries) {
+        throw new IOException("Retry attempted " + count
+            + " times without completing, bailing out.");
+      }
+      count++;
+
+      // Try bulk load
+      loadHFiles.loadHFileQueue(table, connection, queue, startEndKeys);
+    }
+  }
+
+  private void cleanup(String stagingDir, Table table) {
+    // Release the file system delegation token
+    fsDelegationToken.releaseDelegationToken();
+    // Delete the staging directory
+    if (stagingDir != null) {
+      try {
+        sinkFs.delete(new Path(stagingDir), true);
+      } catch (IOException e) {
+        LOG.warn("Failed to delete the staging directory " + stagingDir, e);
+      }
+    }
+    // Do not close the file system
+
+    /*
+     * if (sinkFs != null) { try { sinkFs.close(); } catch (IOException e) { LOG.warn(
+     * "Failed to close the file system"); } }
+     */
+
+    // Close the table
+    if (table != null) {
+      try {
+        table.close();
+      } catch (IOException e) {
+        LOG.warn("Failed to close the table.", e);
+      }
+    }
+  }
+
+  private Map<String, Path> copyHFilesToStagingDir() throws IOException {
+    Map<String, Path> mapOfCopiedHFiles = new HashMap<String, Path>();
+    Pair<byte[], List<String>> familyHFilePathsPair;
+    List<String> hfilePaths;
+    byte[] family;
+    Path familyStagingDir;
+    int familyHFilePathsPairsListSize;
+    int totalNoOfHFiles;
+    List<Pair<byte[], List<String>>> familyHFilePathsPairsList;
+    FileSystem sourceFs = null;
+
+    try {
+      Path sourceClusterPath = new Path(sourceBaseNamespaceDirPath);
+      /*
+       * Path#getFileSystem will by default get the FS from cache. If both source and sink cluster
+       * has same FS name service then it will return peer cluster FS. To avoid this we explicitly
+       * disable the loading of FS from cache, so that a new FS is created with source cluster
+       * configuration.
+       */
+      String sourceScheme = sourceClusterPath.toUri().getScheme();
+      String disableCacheName =
+          String.format("fs.%s.impl.disable.cache", new Object[] { sourceScheme });
+      sourceClusterConf.setBoolean(disableCacheName, true);
+
+      sourceFs = sourceClusterPath.getFileSystem(sourceClusterConf);
+
+      User user = userProvider.getCurrent();
+      // For each table name in the map
+      for (Entry<String, List<Pair<byte[], List<String>>>> tableEntry : bulkLoadHFileMap
+          .entrySet()) {
+        String tableName = tableEntry.getKey();
+
+        // Create staging directory for each table
+        Path stagingDir =
+            createStagingDir(new Path(hbaseStagingDir), user, TableName.valueOf(tableName));
+
+        familyHFilePathsPairsList = tableEntry.getValue();
+        familyHFilePathsPairsListSize = familyHFilePathsPairsList.size();
+
+        // For each list of family hfile paths pair in the table
+        for (int i = 0; i < familyHFilePathsPairsListSize; i++) {
+          familyHFilePathsPair = familyHFilePathsPairsList.get(i);
+
+          family = familyHFilePathsPair.getFirst();
+          hfilePaths = familyHFilePathsPair.getSecond();
+
+          familyStagingDir = new Path(stagingDir, Bytes.toString(family));
+          totalNoOfHFiles = hfilePaths.size();
+
+          // For each list of hfile paths for the family
+          List<Future<Void>> futures = new ArrayList<Future<Void>>();
+          Callable<Void> c;
+          Future<Void> future;
+          int currentCopied = 0;
+          // Copy the hfiles parallely
+          while (totalNoOfHFiles > currentCopied + this.copiesPerThread) {
+            c =
+                new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
+                  currentCopied + this.copiesPerThread));
+            future = exec.submit(c);
+            futures.add(future);
+            currentCopied += this.copiesPerThread;
+          }
+
+          int remaining = totalNoOfHFiles - currentCopied;
+          if (remaining > 0) {
+            c =
+                new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
+                  currentCopied + remaining));
+            future = exec.submit(c);
+            futures.add(future);
+          }
+
+          for (Future<Void> f : futures) {
+            try {
+              f.get();
+            } catch (InterruptedException e) {
+              InterruptedIOException iioe =
+                  new InterruptedIOException(
+                      "Failed to copy HFiles to local file system. This will be retried again "
+                          + "by the source cluster.");
+              iioe.initCause(e);
+              throw iioe;
+            } catch (ExecutionException e) {
+              throw new IOException("Failed to copy HFiles to local file system. This will "
+                  + "be retried again by the source cluster.", e);
+            }
+          }
+        }
+        // Add the staging directory to this table. Staging directory contains all the hfiles
+        // belonging to this table
+        mapOfCopiedHFiles.put(tableName, stagingDir);
+      }
+      return mapOfCopiedHFiles;
+    } finally {
+      if (sourceFs != null) {
+        sourceFs.close();
+      }
+      if(exec != null) {
+        exec.shutdown();
+      }
+    }
+  }
+
+  private Path createStagingDir(Path baseDir, User user, TableName tableName) throws IOException {
+    String tblName = tableName.getNameAsString().replace(":", UNDERSCORE);
+    int RANDOM_WIDTH = 320;
+    int RANDOM_RADIX = 32;
+    String doubleUnderScore = UNDERSCORE + UNDERSCORE;
+    String randomDir = user.getShortName() + doubleUnderScore + tblName + doubleUnderScore
+        + (new BigInteger(RANDOM_WIDTH, new SecureRandom()).toString(RANDOM_RADIX));
+    return createStagingDir(baseDir, user, randomDir);
+  }
+
+  private Path createStagingDir(Path baseDir, User user, String randomDir) throws IOException {
+    Path p = new Path(baseDir, randomDir);
+    sinkFs.mkdirs(p, PERM_ALL_ACCESS);
+    sinkFs.setPermission(p, PERM_ALL_ACCESS);
+    return p;
+  }
+
+  /**
+   * This class will copy the given hfiles from the given source file system to the given local file
+   * system staging directory.
+   */
+  private class Copier implements Callable<Void> {
+    private FileSystem sourceFs;
+    private Path stagingDir;
+    private List<String> hfiles;
+
+    public Copier(FileSystem sourceFs, final Path stagingDir, final List<String> hfiles)
+        throws IOException {
+      this.sourceFs = sourceFs;
+      this.stagingDir = stagingDir;
+      this.hfiles = hfiles;
+    }
+
+    @Override
+    public Void call() throws IOException {
+      Path sourceHFilePath;
+      Path localHFilePath;
+      int totalHFiles = hfiles.size();
+      for (int i = 0; i < totalHFiles; i++) {
+        sourceHFilePath = new Path(sourceBaseNamespaceDirPath, hfiles.get(i));
+        localHFilePath = new Path(stagingDir, sourceHFilePath.getName());
+        try {
+          FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
+          // If any other exception other than FNFE then we will fail the replication requests and
+          // source will retry to replicate these data.
+        } catch (FileNotFoundException e) {
+          LOG.info("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
+              + ". Trying to copy from hfile archive directory.",
+            e);
+          sourceHFilePath = new Path(sourceHFileArchiveDirPath, hfiles.get(i));
+
+          try {
+            FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
+          } catch (FileNotFoundException e1) {
+            // This will mean that the hfile does not exists any where in source cluster FS. So we
+            // cannot do anything here just log and return.
+            LOG.error("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
+                + ". Hence ignoring this hfile from replication..",
+              e1);
+            return null;
+          }
+        }
+        sinkFs.setPermission(localHFilePath, PERM_ALL_ACCESS);
+      }
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
index 37dc1dd..f308daf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSink.java
@@ -47,7 +47,7 @@ public class MetricsSink {
     if (lastTimestampForAge != timestamp) {
       lastTimestampForAge = timestamp;
       age = System.currentTimeMillis() - lastTimestampForAge;
-    } 
+    }
     mss.setLastAppliedOpAge(age);
     return age;
   }
@@ -72,6 +72,17 @@ public class MetricsSink {
   }
 
   /**
+   * Convience method to change metrics when a batch of operations are applied.
+   *
+   * @param batchSize total number of mutations that are applied/replicated
+   * @param hfileSize total number of hfiles that are applied/replicated
+   */
+  public void applyBatch(long batchSize, long hfileSize) {
+    applyBatch(batchSize);
+    mss.incrAppliedHFiles(hfileSize);
+  }
+
+  /**
    * Get the Age of Last Applied Op
    * @return ageOfLastAppliedOp
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
index f9f7001..9687af7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsSource.java
@@ -40,11 +40,13 @@ public class MetricsSource {
   // tracks last shipped timestamp for each wal group
   private Map<String, Long> lastTimeStamps = new HashMap<String, Long>();
   private int lastQueueSize = 0;
+  private long lastHFileRefsQueueSize = 0;
   private String id;
 
   private final MetricsReplicationSourceSource singleSourceSource;
   private final MetricsReplicationSourceSource globalSourceSource;
 
+
   /**
    * Constructor used to register the metrics
    *
@@ -143,6 +145,18 @@ public class MetricsSource {
     globalSourceSource.incrShippedKBs(sizeInKB);
   }
 
+  /**
+   * Convience method to apply changes to metrics do to shipping a batch of logs.
+   *
+   * @param batchSize the size of the batch that was shipped to sinks.
+   * @param hfiles total number of hfiles shipped to sinks.
+   */
+  public void shipBatch(long batchSize, int sizeInKB, long hfiles) {
+    shipBatch(batchSize, sizeInKB);
+    singleSourceSource.incrHFilesShipped(hfiles);
+    globalSourceSource.incrHFilesShipped(hfiles);
+  }
+
   /** increase the byte number read by source from log file */
   public void incrLogReadInBytes(long readInBytes) {
     singleSourceSource.incrLogReadInBytes(readInBytes);
@@ -153,8 +167,10 @@ public class MetricsSource {
   public void clear() {
     singleSourceSource.clear();
     globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
+    globalSourceSource.decrSizeOfHFileRefsQueue(lastHFileRefsQueueSize);
     lastTimeStamps.clear();
     lastQueueSize = 0;
+    lastHFileRefsQueueSize = 0;
   }
 
   /**
@@ -194,4 +210,19 @@ public class MetricsSource {
   public String getPeerID() {
     return id;
   }
+
+  public void incrSizeOfHFileRefsQueue(long size) {
+    singleSourceSource.incrSizeOfHFileRefsQueue(size);
+    globalSourceSource.incrSizeOfHFileRefsQueue(size);
+    lastHFileRefsQueueSize = size;
+  }
+
+  public void decrSizeOfHFileRefsQueue(int size) {
+    singleSourceSource.decrSizeOfHFileRefsQueue(size);
+    globalSourceSource.decrSizeOfHFileRefsQueue(size);
+    lastHFileRefsQueueSize -= size;
+    if (lastHFileRefsQueueSize < 0) {
+      lastHFileRefsQueueSize = 0;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index b3db0f6..30153f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -649,8 +649,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
 
         // set the region name for the target region replica
         Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
-            ReplicationProtbufUtil.buildReplicateWALEntryRequest(
-              entriesArray, location.getRegionInfo().getEncodedNameAsBytes());
+            ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
+                .getRegionInfo().getEncodedNameAsBytes(), null, null, null);
         try {
           PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
           controller.setCallTimeout(timeout);

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index b396dfc..d2a0776 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +45,10 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.wal.WALKey;
@@ -55,6 +59,7 @@ import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
+import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
@@ -71,6 +76,7 @@ public class Replication extends WALActionsListener.Base implements
   private static final Log LOG =
       LogFactory.getLog(Replication.class);
   private boolean replication;
+  private boolean replicationForBulkLoadData;
   private ReplicationSourceManager replicationManager;
   private ReplicationQueues replicationQueues;
   private ReplicationPeers replicationPeers;
@@ -84,7 +90,6 @@ public class Replication extends WALActionsListener.Base implements
   private int statsThreadPeriod;
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
-
   /**
    * Instantiate the replication management (if rep is enabled).
    * @param server Hosting server
@@ -109,11 +114,20 @@ public class Replication extends WALActionsListener.Base implements
     this.server = server;
     this.conf = this.server.getConfiguration();
     this.replication = isReplication(this.conf);
+    this.replicationForBulkLoadData = isReplicationForBulkLoadDataEnabled(this.conf);
     this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
       new ThreadFactoryBuilder()
         .setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
         .setDaemon(true)
         .build());
+    if (this.replicationForBulkLoadData) {
+      if (conf.get(HConstants.REPLICATION_CLUSTER_ID) == null
+          || conf.get(HConstants.REPLICATION_CLUSTER_ID).isEmpty()) {
+        throw new IllegalArgumentException(HConstants.REPLICATION_CLUSTER_ID
+            + " cannot be null/empty when " + HConstants.REPLICATION_BULKLOAD_ENABLE_KEY
+            + " is set to true.");
+      }
+    }
     if (replication) {
       try {
         this.replicationQueues =
@@ -158,6 +172,15 @@ public class Replication extends WALActionsListener.Base implements
     return c.getBoolean(REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
   }
 
+  /**
+   * @param c Configuration to look at
+   * @return True if replication for bulk load data is enabled.
+   */
+  public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
+    return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+  }
+
    /*
     * Returns an object to listen to new wal changes
     **/
@@ -187,14 +210,22 @@ public class Replication extends WALActionsListener.Base implements
   /**
    * Carry on the list of log entries down to the sink
    * @param entries list of entries to replicate
-   * @param cells The data -- the cells -- that <code>entries</code> describes (the entries
-   * do not contain the Cells we are replicating; they are passed here on the side in this
-   * CellScanner).
+   * @param cells The data -- the cells -- that <code>entries</code> describes (the entries do not
+   *          contain the Cells we are replicating; they are passed here on the side in this
+   *          CellScanner).
+   * @param replicationClusterId Id which will uniquely identify source cluster FS client
+   *          configurations in the replication configuration directory
+   * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
+   *          directory required for replicating hfiles
+   * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
    * @throws IOException
    */
-  public void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException {
+  public void replicateLogEntries(List<WALEntry> entries, CellScanner cells,
+      String replicationClusterId, String sourceBaseNamespaceDirPath,
+      String sourceHFileArchiveDirPath) throws IOException {
     if (this.replication) {
-      this.replicationSink.replicateEntries(entries, cells);
+      this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
+        sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath);
     }
   }
 
@@ -226,34 +257,44 @@ public class Replication extends WALActionsListener.Base implements
   }
 
   @Override
-  public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey,
-                                       WALEdit logEdit) {
-    scopeWALEdits(htd, logKey, logEdit);
+  public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
+      throws IOException {
+    scopeWALEdits(htd, logKey, logEdit, this.conf, this.getReplicationManager());
   }
 
   /**
-   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys
-   * from compaction WAL edits and if the scope is local.
+   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
+   * compaction WAL edits and if the scope is local.
    * @param htd Descriptor used to find the scope to use
    * @param logKey Key that may get scoped according to its edits
    * @param logEdit Edits used to lookup the scopes
+   * @param replicationManager Manager used to add bulk load events hfile references
+   * @throws IOException If failed to parse the WALEdit
    */
-  public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey,
-                                   WALEdit logEdit) {
-    NavigableMap<byte[], Integer> scopes =
-        new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+  public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit,
+      Configuration conf, ReplicationSourceManager replicationManager) throws IOException {
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
     byte[] family;
+    boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
     for (Cell cell : logEdit.getCells()) {
-      family = CellUtil.cloneFamily(cell);
-      // This is expected and the KV should not be replicated
-      if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) continue;
-      // Unexpected, has a tendency to happen in unit tests
-      assert htd.getFamily(family) != null;
+      if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        if (replicationForBulkLoadEnabled && CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
+          scopeBulkLoadEdits(htd, replicationManager, scopes, logKey.getTablename(), cell);
+        } else {
+          // Skip the flush/compaction/region events
+          continue;
+        }
+      } else {
+        family = CellUtil.cloneFamily(cell);
+        // Unexpected, has a tendency to happen in unit tests
+        assert htd.getFamily(family) != null;
 
-      int scope = htd.getFamily(family).getScope();
-      if (scope != REPLICATION_SCOPE_LOCAL &&
-          !scopes.containsKey(family)) {
-        scopes.put(family, scope);
+        if (!scopes.containsKey(family)) {
+          int scope = htd.getFamily(family).getScope();
+          if (scope != REPLICATION_SCOPE_LOCAL) {
+            scopes.put(family, scope);
+          }
+        }
       }
     }
     if (!scopes.isEmpty()) {
@@ -261,6 +302,40 @@ public class Replication extends WALActionsListener.Base implements
     }
   }
 
+  private static void scopeBulkLoadEdits(HTableDescriptor htd,
+      ReplicationSourceManager replicationManager, NavigableMap<byte[], Integer> scopes,
+      TableName tableName, Cell cell) throws IOException {
+    byte[] family;
+    try {
+      BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
+      for (StoreDescriptor s : bld.getStoresList()) {
+        family = s.getFamilyName().toByteArray();
+        if (!scopes.containsKey(family)) {
+          int scope = htd.getFamily(family).getScope();
+          if (scope != REPLICATION_SCOPE_LOCAL) {
+            scopes.put(family, scope);
+            addHFileRefsToQueue(replicationManager, tableName, family, s);
+          }
+        } else {
+          addHFileRefsToQueue(replicationManager, tableName, family, s);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get bulk load events information from the wal file.", e);
+      throw e;
+    }
+  }
+
+  private static void addHFileRefsToQueue(ReplicationSourceManager replicationManager,
+      TableName tableName, byte[] family, StoreDescriptor s) throws IOException {
+    try {
+      replicationManager.addHFileRefs(tableName, family, s.getStoreFileList());
+    } catch (ReplicationException e) {
+      LOG.error("Failed to create hfile references in ZK.", e);
+      throw new IOException(e);
+    }
+  }
+
   @Override
   public void preLogRoll(Path oldPath, Path newPath) throws IOException {
     getReplicationManager().preLogRoll(newPath);
@@ -272,8 +347,7 @@ public class Replication extends WALActionsListener.Base implements
   }
 
   /**
-   * This method modifies the master's configuration in order to inject
-   * replication-related features
+   * This method modifies the master's configuration in order to inject replication-related features
    * @param conf
    */
   public static void decorateMasterConfiguration(Configuration conf) {
@@ -285,6 +359,13 @@ public class Replication extends WALActionsListener.Base implements
     if (!plugins.contains(cleanerClass)) {
       conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
     }
+    if (isReplicationForBulkLoadDataEnabled(conf)) {
+      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
+      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
+      if (!plugins.contains(cleanerClass)) {
+        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
+      }
+    }
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/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 f10f5e3..9e7b3af 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
@@ -33,15 +33,16 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -51,6 +52,11 @@ import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * <p>
@@ -78,6 +84,9 @@ public class ReplicationSink {
   private final MetricsSink metrics;
   private final AtomicLong totalReplicatedEdits = new AtomicLong();
   private final Object sharedHtableConLock = new Object();
+  // Number of hfiles that we successfully replicated
+  private long hfilesReplicated = 0;
+  private SourceFSConfigurationProvider provider;
 
   /**
    * Create a sink for replication
@@ -91,6 +100,18 @@ public class ReplicationSink {
     this.conf = HBaseConfiguration.create(conf);
     decorateConf();
     this.metrics = new MetricsSink();
+
+    String className =
+        conf.get("hbase.replication.source.fs.conf.provider",
+          DefaultSourceFSConfigurationProvider.class.getCanonicalName());
+    try {
+      @SuppressWarnings("rawtypes")
+      Class c = Class.forName(className);
+      this.provider = (SourceFSConfigurationProvider) c.newInstance();
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Configured source fs configuration provider class "
+          + className + " throws error.", e);
+    }
   }
 
   /**
@@ -113,9 +134,16 @@ public class ReplicationSink {
    * operates against raw protobuf type saving on a conversion from pb to pojo.
    * @param entries
    * @param cells
-   * @throws IOException
+   * @param replicationClusterId Id which will uniquely identify source cluster FS client
+   *          configurations in the replication configuration directory
+   * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
+   *          directory
+   * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
+   * @throws IOException If failed to replicate the data
    */
-  public void replicateEntries(List<WALEntry> entries, final CellScanner cells) throws IOException {
+  public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
+      String replicationClusterId, String sourceBaseNamespaceDirPath,
+      String sourceHFileArchiveDirPath) throws IOException {
     if (entries.isEmpty()) return;
     if (cells == null) throw new NullPointerException("TODO: Add handling of null CellScanner");
     // Very simple optimization where we batch sequences of rows going
@@ -126,6 +154,10 @@ public class ReplicationSink {
       // invocation of this method per table and cluster id.
       Map<TableName, Map<List<UUID>, List<Row>>> rowMap =
           new TreeMap<TableName, Map<List<UUID>, List<Row>>>();
+
+      // Map of table name Vs list of pair of family and list of hfile paths from its namespace
+      Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = null;
+
       for (WALEntry entry : entries) {
         TableName table =
             TableName.valueOf(entry.getKey().getTableName().toByteArray());
@@ -138,33 +170,60 @@ public class ReplicationSink {
             throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
           }
           Cell cell = cells.current();
-          if (isNewRowOrType(previousCell, cell)) {
-            // Create new mutation
-            m = CellUtil.isDelete(cell)?
-              new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()):
-              new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
-            List<UUID> clusterIds = new ArrayList<UUID>();
-            for(HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()){
-              clusterIds.add(toUUID(clusterId));
+          // Handle bulk load hfiles replication
+          if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
+            if (bulkLoadHFileMap == null) {
+              bulkLoadHFileMap = new HashMap<String, List<Pair<byte[], List<String>>>>();
             }
-            m.setClusterIds(clusterIds);
-            addToHashMultiMap(rowMap, table, clusterIds, m);
-          }
-          if (CellUtil.isDelete(cell)) {
-            ((Delete)m).addDeleteMarker(cell);
+            buildBulkLoadHFileMap(bulkLoadHFileMap, table, cell);
           } else {
-            ((Put)m).add(cell);
+            // Handle wal replication
+            if (isNewRowOrType(previousCell, cell)) {
+              // Create new mutation
+              m =
+                  CellUtil.isDelete(cell) ? new Delete(cell.getRowArray(), cell.getRowOffset(),
+                      cell.getRowLength()) : new Put(cell.getRowArray(), cell.getRowOffset(),
+                      cell.getRowLength());
+              List<UUID> clusterIds = new ArrayList<UUID>();
+              for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {
+                clusterIds.add(toUUID(clusterId));
+              }
+              m.setClusterIds(clusterIds);
+              addToHashMultiMap(rowMap, table, clusterIds, m);
+            }
+            if (CellUtil.isDelete(cell)) {
+              ((Delete) m).addDeleteMarker(cell);
+            } else {
+              ((Put) m).add(cell);
+            }
+            previousCell = cell;
           }
-          previousCell = cell;
         }
         totalReplicated++;
       }
-      for (Entry<TableName, Map<List<UUID>,List<Row>>> entry : rowMap.entrySet()) {
-        batch(entry.getKey(), entry.getValue().values());
+
+      // TODO Replicating mutations and bulk loaded data can be made parallel
+      if (!rowMap.isEmpty()) {
+        LOG.debug("Started replicating mutations.");
+        for (Entry<TableName, Map<List<UUID>, List<Row>>> entry : rowMap.entrySet()) {
+          batch(entry.getKey(), entry.getValue().values());
+        }
+        LOG.debug("Finished replicating mutations.");
+      }
+
+      if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
+        LOG.debug("Started replicating bulk loaded data.");
+        HFileReplicator hFileReplicator =
+            new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
+                sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,
+                getConnection());
+        hFileReplicator.replicate();
+        LOG.debug("Finished replicating bulk loaded data.");
       }
+
       int size = entries.size();
       this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
-      this.metrics.applyBatch(size);
+      this.metrics.applyBatch(size + hfilesReplicated, hfilesReplicated);
       this.totalReplicatedEdits.addAndGet(totalReplicated);
     } catch (IOException ex) {
       LOG.error("Unable to accept edit because:", ex);
@@ -172,6 +231,76 @@ public class ReplicationSink {
     }
   }
 
+  private void buildBulkLoadHFileMap(
+      final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, TableName table,
+      Cell cell) throws IOException {
+    BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
+    List<StoreDescriptor> storesList = bld.getStoresList();
+    int storesSize = storesList.size();
+    for (int j = 0; j < storesSize; j++) {
+      StoreDescriptor storeDescriptor = storesList.get(j);
+      List<String> storeFileList = storeDescriptor.getStoreFileList();
+      int storeFilesSize = storeFileList.size();
+      hfilesReplicated += storeFilesSize;
+      for (int k = 0; k < storeFilesSize; k++) {
+        byte[] family = storeDescriptor.getFamilyName().toByteArray();
+
+        // Build hfile relative path from its namespace
+        String pathToHfileFromNS = getHFilePath(table, bld, storeFileList.get(k), family);
+
+        String tableName = table.getNameWithNamespaceInclAsString();
+        if (bulkLoadHFileMap.containsKey(tableName)) {
+          List<Pair<byte[], List<String>>> familyHFilePathsList = bulkLoadHFileMap.get(tableName);
+          boolean foundFamily = false;
+          for (int i = 0; i < familyHFilePathsList.size(); i++) {
+            Pair<byte[], List<String>> familyHFilePathsPair = familyHFilePathsList.get(i);
+            if (Bytes.equals(familyHFilePathsPair.getFirst(), family)) {
+              // Found family already present, just add the path to the existing list
+              familyHFilePathsPair.getSecond().add(pathToHfileFromNS);
+              foundFamily = true;
+              break;
+            }
+          }
+          if (!foundFamily) {
+            // Family not found, add this family and its hfile paths pair to the list
+            addFamilyAndItsHFilePathToTableInMap(family, pathToHfileFromNS, familyHFilePathsList);
+          }
+        } else {
+          // Add this table entry into the map
+          addNewTableEntryInMap(bulkLoadHFileMap, family, pathToHfileFromNS, tableName);
+        }
+      }
+    }
+  }
+
+  private void addFamilyAndItsHFilePathToTableInMap(byte[] family, String pathToHfileFromNS,
+      List<Pair<byte[], List<String>>> familyHFilePathsList) {
+    List<String> hfilePaths = new ArrayList<String>();
+    hfilePaths.add(pathToHfileFromNS);
+    familyHFilePathsList.add(new Pair<byte[], List<String>>(family, hfilePaths));
+  }
+
+  private void addNewTableEntryInMap(
+      final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, byte[] family,
+      String pathToHfileFromNS, String tableName) {
+    List<String> hfilePaths = new ArrayList<String>();
+    hfilePaths.add(pathToHfileFromNS);
+    Pair<byte[], List<String>> newFamilyHFilePathsPair =
+        new Pair<byte[], List<String>>(family, hfilePaths);
+    List<Pair<byte[], List<String>>> newFamilyHFilePathsList =
+        new ArrayList<Pair<byte[], List<String>>>();
+    newFamilyHFilePathsList.add(newFamilyHFilePathsPair);
+    bulkLoadHFileMap.put(tableName, newFamilyHFilePathsList);
+  }
+
+  private String getHFilePath(TableName table, BulkLoadDescriptor bld, String storeFile,
+      byte[] family) {
+    return new StringBuilder(100).append(table.getNamespaceAsString()).append(Path.SEPARATOR)
+        .append(table.getQualifierAsString()).append(Path.SEPARATOR)
+        .append(Bytes.toString(bld.getEncodedRegionName().toByteArray())).append(Path.SEPARATOR)
+        .append(Bytes.toString(family)).append(Path.SEPARATOR).append(storeFile).toString();
+  }
+
   /**
    * @param previousCell
    * @param cell
@@ -241,22 +370,13 @@ public class ReplicationSink {
     }
     Table table = null;
     try {
-      // See https://en.wikipedia.org/wiki/Double-checked_locking
-      Connection connection = this.sharedHtableCon;
-      if (connection == null) {
-        synchronized (sharedHtableConLock) {
-          connection = this.sharedHtableCon;
-          if (connection == null) {
-            connection = this.sharedHtableCon = ConnectionFactory.createConnection(this.conf);
-          }
-        }
-      }
+      Connection connection = getConnection();
       table = connection.getTable(tableName);
       for (List<Row> rows : allRows) {
         table.batch(rows, null);
       }
     } catch (InterruptedException ix) {
-      throw (InterruptedIOException)new InterruptedIOException().initCause(ix);
+      throw (InterruptedIOException) new InterruptedIOException().initCause(ix);
     } finally {
       if (table != null) {
         table.close();
@@ -264,6 +384,20 @@ public class ReplicationSink {
     }
   }
 
+  private Connection getConnection() throws IOException {
+    // See https://en.wikipedia.org/wiki/Double-checked_locking
+    Connection connection = sharedHtableCon;
+    if (connection == null) {
+      synchronized (sharedHtableConLock) {
+        connection = sharedHtableCon;
+        if (connection == null) {
+          connection = sharedHtableCon = ConnectionFactory.createConnection(conf);
+        }
+      }
+    }
+    return connection;
+  }
+
   /**
    * Get a string representation of this sink's metrics
    * @return string with the total replicated edits count and the date

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 3d99523..868ddee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -37,7 +37,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,9 +46,10 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -59,8 +59,12 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -223,6 +227,34 @@ public class ReplicationSource extends Thread
     }
   }
 
+  @Override
+  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+      throws ReplicationException {
+    String peerId = peerClusterZnode;
+    if (peerId.contains("-")) {
+      // peerClusterZnode will be in the form peerId + "-" + rsZNode.
+      // A peerId will not have "-" in its name, see HBASE-11394
+      peerId = peerClusterZnode.split("-")[0];
+    }
+    Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
+    if (tableCFMap != null) {
+      List<String> tableCfs = tableCFMap.get(tableName);
+      if (tableCFMap.containsKey(tableName)
+          && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
+        this.replicationQueues.addHFileRefs(peerId, files);
+        metrics.incrSizeOfHFileRefsQueue(files.size());
+      } else {
+        LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
+            + Bytes.toString(family) + " to peer id " + peerId);
+      }
+    } else {
+      // user has explicitly not defined any table cfs for replication, means replicate all the
+      // data
+      this.replicationQueues.addHFileRefs(peerId, files);
+      metrics.incrSizeOfHFileRefsQueue(files.size());
+    }
+  }
+
   private void uninitialize() {
     LOG.debug("Source exiting " + this.peerId);
     metrics.clear();
@@ -471,6 +503,8 @@ public class ReplicationSource extends Thread
     private int currentSize = 0;
     // Indicates whether this particular worker is running
     private boolean workerRunning = true;
+    // Current number of hfiles that we need to replicate
+    private long currentNbHFiles = 0;
 
     public ReplicationSourceWorkerThread(String walGroupId,
         PriorityBlockingQueue<Path> queue, ReplicationQueueInfo replicationQueueInfo, ReplicationSource source) {
@@ -550,6 +584,7 @@ public class ReplicationSource extends Thread
 
         boolean gotIOE = false;
         currentNbOperations = 0;
+        currentNbHFiles = 0;
         List<WAL.Entry> entries = new ArrayList<WAL.Entry>(1);
         currentSize = 0;
         try {
@@ -701,6 +736,28 @@ public class ReplicationSource extends Thread
       return seenEntries == 0 && processEndOfFile();
     }
 
+    private void cleanUpHFileRefs(WALEdit edit) throws IOException {
+      String peerId = peerClusterZnode;
+      if (peerId.contains("-")) {
+        // peerClusterZnode will be in the form peerId + "-" + rsZNode.
+        // A peerId will not have "-" in its name, see HBASE-11394
+        peerId = peerClusterZnode.split("-")[0];
+      }
+      List<Cell> cells = edit.getCells();
+      for (int i = 0; i < cells.size(); i++) {
+        Cell cell = cells.get(i);
+        if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
+          BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
+          List<StoreDescriptor> stores = bld.getStoresList();
+          for (int j = 0; j < stores.size(); j++) {
+            List<String> storeFileList = stores.get(j).getStoreFileList();
+            manager.cleanUpHFileRefs(peerId, storeFileList);
+            metrics.decrSizeOfHFileRefsQueue(storeFileList.size());
+          }
+        }
+      }
+    }
+
     /**
      * Poll for the next path
      * @return true if a path was obtained, false if not
@@ -853,14 +910,31 @@ public class ReplicationSource extends Thread
     private int countDistinctRowKeys(WALEdit edit) {
       List<Cell> cells = edit.getCells();
       int distinctRowKeys = 1;
+      int totalHFileEntries = 0;
       Cell lastCell = cells.get(0);
+
       for (int i = 0; i < edit.size(); i++) {
+        // Count HFiles to be replicated
+        if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
+          try {
+            BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
+            List<StoreDescriptor> stores = bld.getStoresList();
+            for (int j = 0; j < stores.size(); j++) {
+              totalHFileEntries += stores.get(j).getStoreFileList().size();
+            }
+          } catch (IOException e) {
+            LOG.error("Failed to deserialize bulk load entry from wal edit. "
+                + "This its hfiles count will not be added into metric.");
+          }
+        }
+
         if (!CellUtil.matchingRows(cells.get(i), lastCell)) {
           distinctRowKeys++;
         }
         lastCell = cells.get(i);
       }
-      return distinctRowKeys;
+      currentNbHFiles += totalHFileEntries;
+      return distinctRowKeys + totalHFileEntries;
     }
 
     /**
@@ -914,6 +988,12 @@ public class ReplicationSource extends Thread
           }
 
           if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
+            //Clean up hfile references
+            int size = entries.size();
+            for (int i = 0; i < size; i++) {
+              cleanUpHFileRefs(entries.get(i).getEdit());
+            }
+            //Log and clean up WAL logs
             manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,
               this.repLogReader.getPosition(), this.replicationQueueInfo.isQueueRecovered(),
               currentWALisBeingWrittenTo);
@@ -925,7 +1005,7 @@ public class ReplicationSource extends Thread
           totalReplicatedEdits.addAndGet(entries.size());
           totalReplicatedOperations.addAndGet(currentNbOperations);
           // FIXME check relationship between wal group and overall
-          metrics.shipBatch(currentNbOperations, currentSize / 1024);
+          metrics.shipBatch(currentNbOperations, currentSize / 1024, currentNbHFiles);
           metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
             walGroupId);
           if (LOG.isTraceEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index 1e9c714..7f4a9f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.UUID;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -26,7 +27,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 
@@ -105,4 +108,14 @@ public interface ReplicationSourceInterface {
    */
   String getStats();
 
+  /**
+   * Add hfile names to the queue to be replicated.
+   * @param tableName Name of the table these files belongs to
+   * @param family Name of the family these files belong to
+   * @param files files whose names needs to be added to the queue to be replicated
+   * @throws ReplicationException If failed to add hfile references
+   */
+  void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+      throws ReplicationException;
+
 }


[12/26] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Posted by sy...@apache.org.
HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin


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

Branch: refs/heads/hbase-12439
Commit: bebcc09fb392b3494131c792520406c001dbd511
Parents: 9511150
Author: stack <st...@apache.org>
Authored: Thu Dec 10 15:02:21 2015 -0800
Committer: stack <st...@apache.org>
Committed: Thu Dec 10 15:02:21 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |    8 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 1417 +-----------------
 .../hbase/IntegrationTestDDLMasterFailover.java |    2 +-
 .../hbase/IntegrationTestIngestWithMOB.java     |    2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   24 +-
 .../apache/hadoop/hbase/TestAcidGuarantees.java |    2 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |    4 +-
 .../hbase/client/TestMetaWithReplicas.java      |    8 +-
 .../hbase/mob/TestExpiredMobFileCleaner.java    |    2 +-
 .../compactions/TestFIFOCompactionPolicy.java   |    8 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   88 +-
 hbase-shell/src/main/ruby/hbase/security.rb     |    6 +-
 .../src/main/ruby/hbase/visibility_labels.rb    |    2 +-
 hbase-shell/src/test/ruby/hbase/admin_test.rb   |   14 +-
 src/main/asciidoc/_chapters/cp.adoc             |    2 +-
 src/main/asciidoc/_chapters/external_apis.adoc  |    4 +-
 src/main/asciidoc/_chapters/ops_mgt.adoc        |    2 +-
 src/main/asciidoc/_chapters/schema_design.adoc  |    2 +-
 18 files changed, 159 insertions(+), 1438 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 290ecb5..b06902a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -487,7 +487,10 @@ public interface Admin extends Abortable, Closeable {
    * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are
    * yet to be updated Pair.getSecond() is the total number of regions of the table
    * @throws IOException if a remote or network exception occurs
+   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getAlterStatus(TableName)}
+   *     instead.
    */
+  @Deprecated
   Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException;
 
   /**
@@ -1181,9 +1184,8 @@ public interface Admin extends Abortable, Closeable {
       throws IOException, SnapshotCreationException, IllegalArgumentException;
 
   /**
-   * public void snapshot(final String snapshotName, Create a timestamp consistent snapshot for the
-   * given table. final byte[] tableName) throws IOException, Snapshots are considered unique based
-   * on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even a
+   * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique
+   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
    * different type or with different parameters) will fail with a {@link SnapshotCreationException}
    * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
    * HBase.


[13/26] hbase git commit: HBASE-14906 Improvements on FlushLargeStoresPolicy (Yu Li)

Posted by sy...@apache.org.
HBASE-14906 Improvements on FlushLargeStoresPolicy (Yu Li)


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

Branch: refs/heads/hbase-12439
Commit: c15e0af84aeb4ab992482a957c2b242d2ab57d76
Parents: bebcc09
Author: stack <st...@apache.org>
Authored: Thu Dec 10 16:49:23 2015 -0800
Committer: stack <st...@apache.org>
Committed: Thu Dec 10 16:49:23 2015 -0800

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        | 17 ++++----
 .../regionserver/FlushLargeStoresPolicy.java    | 44 ++++++++++++++------
 .../hadoop/hbase/regionserver/HRegion.java      |  4 ++
 .../regionserver/TestPerColumnFamilyFlush.java  | 10 +++--
 4 files changed, 51 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c15e0af8/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index fa5d522..37a6298 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -612,16 +612,17 @@ possible configurations would overwhelm and obscure the important.
     every hbase.server.thread.wakefrequency.</description>
   </property>
   <property>
-    <name>hbase.hregion.percolumnfamilyflush.size.lower.bound</name>
+    <name>hbase.hregion.percolumnfamilyflush.size.lower.bound.min</name>
     <value>16777216</value>
     <description>
-    If FlushLargeStoresPolicy is used, then every time that we hit the
-    total memstore limit, we find out all the column families whose memstores
-    exceed this value, and only flush them, while retaining the others whose
-    memstores are lower than this limit. If none of the families have their
-    memstore size more than this, all the memstores will be flushed
-    (just as usual). This value should be less than half of the total memstore
-    threshold (hbase.hregion.memstore.flush.size).
+    If FlushLargeStoresPolicy is used and there are multiple column families,
+    then every time that we hit the total memstore limit, we find out all the
+    column families whose memstores exceed a "lower bound" and only flush them
+    while retaining the others in memory. The "lower bound" will be
+    "hbase.hregion.memstore.flush.size / column_family_number" by default
+    unless value of this property is larger than that. If none of the families
+    have their memstore size more than lower bound, all the memstores will be
+    flushed (just as usual).
     </description>
   </property>
   <property>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c15e0af8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
index 328e890..b4d47c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
@@ -38,35 +38,50 @@ public class FlushLargeStoresPolicy extends FlushPolicy {
   public static final String HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND =
       "hbase.hregion.percolumnfamilyflush.size.lower.bound";
 
-  private static final long DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND = 1024 * 1024 * 16L;
+  public static final String HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN =
+      "hbase.hregion.percolumnfamilyflush.size.lower.bound.min";
 
-  private long flushSizeLowerBound;
+  private static final long DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN =
+      1024 * 1024 * 16L;
+
+  private long flushSizeLowerBound = -1;
 
   @Override
   protected void configureForRegion(HRegion region) {
     super.configureForRegion(region);
-    long flushSizeLowerBound;
+    int familyNumber = region.getTableDesc().getFamilies().size();
+    if (familyNumber <= 1) {
+      // No need to parse and set flush size lower bound if only one family
+      // Family number might also be zero in some of our unit test case
+      return;
+    }
+    // For multiple families, lower bound is the "average flush size" by default
+    // unless setting in configuration is larger.
+    long flushSizeLowerBound = region.getMemstoreFlushSize() / familyNumber;
+    long minimumLowerBound =
+        getConf().getLong(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
+          DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN);
+    if (minimumLowerBound > flushSizeLowerBound) {
+      flushSizeLowerBound = minimumLowerBound;
+    }
+    // use the setting in table description if any
     String flushedSizeLowerBoundString =
         region.getTableDesc().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
     if (flushedSizeLowerBoundString == null) {
-      flushSizeLowerBound =
-          getConf().getLong(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
-            DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
       if (LOG.isDebugEnabled()) {
-        LOG.debug(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
-            + " is not specified, use global config(" + flushSizeLowerBound + ") instead");
+        LOG.debug("No " + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
+            + " set in description of table " + region.getTableDesc().getTableName()
+            + ", use config (" + flushSizeLowerBound + ") instead");
       }
     } else {
       try {
         flushSizeLowerBound = Long.parseLong(flushedSizeLowerBoundString);
       } catch (NumberFormatException nfe) {
-        flushSizeLowerBound =
-            getConf().getLong(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
-              DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
+        // fall back for fault setting
         LOG.warn("Number format exception when parsing "
             + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND + " for table "
             + region.getTableDesc().getTableName() + ":" + flushedSizeLowerBoundString + ". " + nfe
-            + ", use global config(" + flushSizeLowerBound + ") instead");
+            + ", use config (" + flushSizeLowerBound + ") instead");
 
       }
     }
@@ -87,6 +102,11 @@ public class FlushLargeStoresPolicy extends FlushPolicy {
 
   @Override
   public Collection<Store> selectStoresToFlush() {
+    // no need to select stores if only one family
+    if (region.getTableDesc().getFamilies().size() == 1) {
+      return region.stores.values();
+    }
+    // start selection
     Collection<Store> stores = region.stores.values();
     Set<Store> specificStoresToFlush = new HashSet<Store>();
     for (Store store : stores) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c15e0af8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 484d5ee..9549a13 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -8183,4 +8183,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return this.getRegionInfo().isMetaRegion() ? CellComparator.META_COMPARATOR
         : CellComparator.COMPARATOR;
   }
+
+  public long getMemstoreFlushSize() {
+    return this.memstoreFlushSize;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c15e0af8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
index 0df2799..624f4a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
@@ -128,7 +128,8 @@ public class TestPerColumnFamilyFlush {
     Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND, 100 * 1024);
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
+      100 * 1024);
     // Intialize the region
     Region region = initHRegion("testSelectiveFlushWhenEnabled", conf);
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
@@ -336,7 +337,7 @@ public class TestPerColumnFamilyFlush {
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 20000);
     // Carefully chosen limits so that the memstore just flushes when we're done
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND, 10000);
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 10000);
     final int numRegionServers = 4;
     try {
       TEST_UTIL.startMiniCluster(numRegionServers);
@@ -451,7 +452,7 @@ public class TestPerColumnFamilyFlush {
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 128 * 1024 * 1024);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
     long cfFlushSizeLowerBound = 2048;
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
       cfFlushSizeLowerBound);
 
     // One hour, prevent periodic rolling
@@ -568,7 +569,6 @@ public class TestPerColumnFamilyFlush {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, memstoreFlushSize);
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllStoresPolicy.class.getName());
-    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND, 400 * 1024);
     conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000);
     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
       ConstantSizeRegionSplitPolicy.class.getName());
@@ -608,6 +608,8 @@ public class TestPerColumnFamilyFlush {
 
     LOG.info("==============Test with selective flush enabled===============");
     conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
+    // default value of per-cf flush lower bound is too big, set to a small enough value
+    conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 0);
     try {
       TEST_UTIL.startMiniCluster(1);
       TEST_UTIL.getHBaseAdmin().createNamespace(


[21/26] hbase git commit: HBASE-14895 Seek only to the newly flushed file on scanner reset on flush (Ram)

Posted by sy...@apache.org.
HBASE-14895 Seek only to the newly flushed file on scanner reset on flush
(Ram)


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

Branch: refs/heads/hbase-12439
Commit: 555d9b70bd650a0df0ed9e382de449c337274493
Parents: 676ce01
Author: ramkrishna <ra...@gmail.com>
Authored: Mon Dec 14 10:09:41 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Mon Dec 14 10:13:53 2015 +0530

----------------------------------------------------------------------
 .../regionserver/ChangedReadersObserver.java    |   3 +-
 .../hadoop/hbase/regionserver/HStore.java       |  30 ++++-
 .../regionserver/ReversedStoreScanner.java      |   6 +-
 .../apache/hadoop/hbase/regionserver/Store.java |  19 +++
 .../hbase/regionserver/StoreFileScanner.java    |   3 +
 .../hadoop/hbase/regionserver/StoreScanner.java | 116 +++++++++-------
 .../client/TestBlockEvictionFromClient.java     | 132 +++++++++++++++++++
 .../hbase/regionserver/TestStoreScanner.java    |   4 +-
 .../hbase/regionserver/TestWideScanner.java     |   2 +-
 9 files changed, 258 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
index 36b7559..0bc75e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -33,5 +34,5 @@ public interface ChangedReadersObserver {
    * Notify observers.
    * @throws IOException e
    */
-  void updateReaders() throws IOException;
+  void updateReaders(List<StoreFile> sfs) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 49b6c50..badbd65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1042,7 +1042,7 @@ public class HStore implements Store {
       this.lock.writeLock().unlock();
     }
     // notify to be called here - only in case of flushes
-    notifyChangedReadersObservers();
+    notifyChangedReadersObservers(sfs);
     if (LOG.isTraceEnabled()) {
       long totalSize = 0;
       for (StoreFile sf : sfs) {
@@ -1060,9 +1060,9 @@ public class HStore implements Store {
    * Notify all observers that set of Readers has changed.
    * @throws IOException
    */
-  private void notifyChangedReadersObservers() throws IOException {
+  private void notifyChangedReadersObservers(List<StoreFile> sfs) throws IOException {
     for (ChangedReadersObserver o : this.changedReaderObservers) {
-      o.updateReaders();
+      o.updateReaders(sfs);
     }
   }
 
@@ -1102,6 +1102,30 @@ public class HStore implements Store {
   }
 
   @Override
+  public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
+      boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
+      byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException {
+    List<KeyValueScanner> memStoreScanners = null;
+    if (includeMemstoreScanner) {
+      this.lock.readLock().lock();
+      try {
+        memStoreScanners = this.memstore.getScanners(readPt);
+      } finally {
+        this.lock.readLock().unlock();
+      }
+    }
+    List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(files,
+      cacheBlocks, usePread, isCompaction, false, matcher, readPt, isPrimaryReplicaStore());
+    List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(sfScanners.size() + 1);
+    scanners.addAll(sfScanners);
+    // Then the memstore scanners
+    if (memStoreScanners != null) {
+      scanners.addAll(memStoreScanners);
+    }
+    return scanners;
+  }
+
+  @Override
   public void addChangedReaderObserver(ChangedReadersObserver o) {
     this.changedReaderObservers.add(o);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
index 0e1d90f..41c13f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
@@ -123,13 +123,15 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
 
   @Override
   public boolean seekToPreviousRow(Cell key) throws IOException {
-    checkReseek();
+    boolean flushed = checkFlushed();
+    checkReseek(flushed);
     return this.heap.seekToPreviousRow(key);
   }
   
   @Override
   public boolean backwardSeek(Cell key) throws IOException {
-    checkReseek();
+    boolean flushed = checkFlushed();
+    checkReseek(flushed);
     return this.heap.backwardSeek(key);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index f137a8e..8bb10f0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -105,6 +105,25 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
     byte[] stopRow,
     long readPt
   ) throws IOException;
+
+  /**
+   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
+   * (that happens further down the line).
+   * @param files the list of files on which the scanners has to be created
+   * @param cacheBlocks cache the blocks or not
+   * @param isGet true if it is get, false if not
+   * @param usePread true to use pread, false if not
+   * @param isCompaction true if the scanner is created for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the start row
+   * @param stopRow the stop row
+   * @param readPt the read point of the current scan
+   * @param includeMemstoreScanner true if memstore has to be included
+   * @return scanners on the given files and on the memstore if specified
+   */
+   List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean isGet,
+          boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
+          byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException;
   
   ScanInfo getScanInfo();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index c864733..d752e17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -48,6 +48,7 @@ public class StoreFileScanner implements KeyValueScanner {
   private final StoreFile.Reader reader;
   private final HFileScanner hfs;
   private Cell cur = null;
+  private boolean closed = false;
 
   private boolean realSeekDone;
   private boolean delayedReseek;
@@ -246,11 +247,13 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public void close() {
+    if (closed) return;
     cur = null;
     this.hfs.close();
     if (this.reader != null) {
       this.reader.decrementRefCount();
     }
+    closed = true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 44f07f7..987a3f5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.NavigableSet;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -126,6 +127,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   private boolean scanUsePread = false;
   // Indicates whether there was flush during the course of the scan
   protected volatile boolean flushed = false;
+  // generally we get one file from a flush
+  protected List<StoreFile> flushedStoreFiles = new ArrayList<StoreFile>(1);
+  // The current list of scanners
+  protected List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
+  // flush update lock
+  private ReentrantLock flushLock = new ReentrantLock();
   
   protected final long readPt;
 
@@ -170,6 +177,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
      }
   }
 
+  protected void addCurrentScanners(List<? extends KeyValueScanner> scanners) {
+    this.currentScanners.addAll(scanners);
+  }
   /**
    * Opens a scanner across memstore, snapshot, and all StoreFiles. Assumes we
    * are not in a compaction.
@@ -207,7 +217,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     // set rowOffset
     this.storeOffset = scan.getRowOffsetPerColumnFamily();
-
+    addCurrentScanners(scanners);
     // Combine all seeked scanners with a heap
     resetKVHeap(scanners, store.getComparator());
   }
@@ -264,7 +274,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
     // Seek all scanners to the initial key
     seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
-
+    addCurrentScanners(scanners);
     // Combine all seeked scanners with a heap
     resetKVHeap(scanners, store.getComparator());
   }
@@ -303,6 +313,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     }
     // Seek all scanners to the initial key
     seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);
+    addCurrentScanners(scanners);
     resetKVHeap(scanners, scanInfo.getComparator());
   }
 
@@ -403,7 +414,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public Cell peek() {
-    checkResetHeap();
+    checkFlushed();
     if (this.heap == null) {
       return this.lastTop;
     }
@@ -435,11 +446,13 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       this.heapsForDelayedClose.clear();
       if (this.heap != null) {
         this.heap.close();
+        this.currentScanners.clear();
         this.heap = null; // CLOSED!
       }
     } else {
       if (this.heap != null) {
         this.heapsForDelayedClose.add(this.heap);
+        this.currentScanners.clear();
         this.heap = null;
       }
     }
@@ -448,9 +461,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public boolean seek(Cell key) throws IOException {
-    checkResetHeap();
+    boolean flushed = checkFlushed();
     // reset matcher state, in case that underlying store changed
-    checkReseek();
+    checkReseek(flushed);
     return this.heap.seek(key);
   }
 
@@ -470,8 +483,8 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (scannerContext == null) {
       throw new IllegalArgumentException("Scanner context cannot be null");
     }
-    checkResetHeap();
-    if (checkReseek()) {
+    boolean flushed = checkFlushed();
+    if (checkReseek(flushed)) {
       return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
     }
 
@@ -665,36 +678,25 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   // Implementation of ChangedReadersObserver
   @Override
-  public void updateReaders() throws IOException {
+  public void updateReaders(List<StoreFile> sfs) throws IOException {
     flushed = true;
+    flushLock.lock();
+    try {
+      flushedStoreFiles.addAll(sfs);
+    } finally {
+      flushLock.unlock();
+    }
     // Let the next() call handle re-creating and seeking
   }
 
-  protected void nullifyCurrentHeap() {
-    if (this.closing) return;
-    // All public synchronized API calls will call 'checkReseek' which will cause
-    // the scanner stack to reseek if this.heap==null && this.lastTop != null.
-    // But if two calls to updateReaders() happen without a 'next' or 'peek' then we
-    // will end up calling this.peek() which would cause a reseek in the middle of a updateReaders
-    // which is NOT what we want, not to mention could cause an NPE. So we early out here.
-    if (this.heap == null) return;
-    // this could be null.
-    this.lastTop = this.heap.peek();
-
-    //DebugPrint.println("SS updateReaders, topKey = " + lastTop);
-
-    // close scanners to old obsolete Store files
-    this.heapsForDelayedClose.add(this.heap);// Don't close now. Delay it till StoreScanner#close
-    this.heap = null; // the re-seeks could be slow (access HDFS) free up memory ASAP
-  }
-
   /**
+   * @param flushed indicates if there was a flush
    * @return true if top of heap has changed (and KeyValueHeap has to try the
    *         next KV)
    * @throws IOException
    */
-  protected boolean checkReseek() throws IOException {
-    if (this.heap == null && this.lastTop != null) {
+  protected boolean checkReseek(boolean flushed) throws IOException {
+    if (flushed && this.lastTop != null) {
       resetScannerStack(this.lastTop);
       if (this.heap.peek() == null
           || store.getComparator().compareRows(this.lastTop, this.heap.peek()) != 0) {
@@ -710,21 +712,37 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   }
 
   protected void resetScannerStack(Cell lastTopKey) throws IOException {
-    if (heap != null) {
-      throw new RuntimeException("StoreScanner.reseek run on an existing heap!");
-    }
-
     /* When we have the scan object, should we not pass it to getScanners()
      * to get a limited set of scanners? We did so in the constructor and we
-     * could have done it now by storing the scan object from the constructor */
-    List<KeyValueScanner> scanners = getScannersNoCompaction();
+     * could have done it now by storing the scan object from the constructor
+     */
 
-    // Seek all scanners to the initial key
-    seekScanners(scanners, lastTopKey, false, parallelSeekEnabled);
+    final boolean isCompaction = false;
+    boolean usePread = get || scanUsePread;
+    List<KeyValueScanner> scanners = null;
+    try {
+      flushLock.lock();
+      scanners = selectScannersFrom(store.getScanners(flushedStoreFiles, cacheBlocks, get, usePread,
+        isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, true));
+      // Clear the current set of flushed store files so that they don't get added again
+      flushedStoreFiles.clear();
+    } finally {
+      flushLock.unlock();
+    }
 
+    // Seek the new scanners to the last key
+    seekScanners(scanners, lastTopKey, false, parallelSeekEnabled);
+    // remove the older memstore scanner
+    for (int i = 0; i < currentScanners.size(); i++) {
+      if (!currentScanners.get(i).isFileScanner()) {
+        currentScanners.remove(i);
+        break;
+      }
+    }
+    // add the newly created scanners on the flushed files and the current active memstore scanner
+    addCurrentScanners(scanners);
     // Combine all seeked scanners with a heap
-    resetKVHeap(scanners, store.getComparator());
-
+    resetKVHeap(this.currentScanners, store.getComparator());
     // Reset the state of the Query Matcher and set to top row.
     // Only reset and call setRow if the row changes; avoids confusing the
     // query matcher if scanning intra-row.
@@ -771,34 +789,36 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public boolean reseek(Cell kv) throws IOException {
-    checkResetHeap();
+    boolean flushed = checkFlushed();
     // Heap will not be null, if this is called from next() which.
     // If called from RegionScanner.reseek(...) make sure the scanner
     // stack is reset if needed.
-    checkReseek();
+    checkReseek(flushed);
     if (explicitColumnQuery && lazySeekEnabledGlobally) {
       return heap.requestSeek(kv, true, useRowColBloom);
     }
     return heap.reseek(kv);
   }
 
-  protected void checkResetHeap() {
+  protected boolean checkFlushed() {
     // check the var without any lock. Suppose even if we see the old
     // value here still it is ok to continue because we will not be resetting
     // the heap but will continue with the referenced memstore's snapshot. For compactions
     // any way we don't need the updateReaders at all to happen as we still continue with 
     // the older files
     if (flushed) {
-      // If the 'flushed' is found to be true then there is a need to ensure
-      // that the current scanner updates the heap that it has and then proceed
-      // with the scan and ensure to reset the flushed inside the lock
-      // One thing can be sure that the same store scanner cannot be in reseek and
-      // next at the same time ie. within the same store scanner it is always single
-      // threaded
-      nullifyCurrentHeap();
+      // If there is a flush and the current scan is notified on the flush ensure that the 
+      // scan's heap gets reset and we do a seek on the newly flushed file.
+      if(!this.closing) {
+        this.lastTop = this.heap.peek();
+      } else {
+        return false;
+      }
       // reset the flag
       flushed = false;
+      return true;
     }
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index 6dedee2..a812623 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -921,6 +921,138 @@ public class TestBlockEvictionFromClient {
   }
 
   @Test
+  public void testBlockEvictionAfterHBASE13082WithCompactionAndFlush()
+      throws IOException, InterruptedException {
+    // do flush and scan in parallel
+    HTable table = null;
+    try {
+      latch = new CountDownLatch(1);
+      compactionLatch = new CountDownLatch(1);
+      TableName tableName =
+          TableName.valueOf("testBlockEvictionAfterHBASE13082WithCompactionAndFlush");
+      // Create a table with block size as 1024
+      table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024,
+          CustomInnerRegionObserverWrapper.class.getName());
+      // get the block cache and region
+      RegionLocator locator = table.getRegionLocator();
+      String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+      Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(
+          regionName);
+      Store store = region.getStores().iterator().next();
+      CacheConfig cacheConf = store.getCacheConfig();
+      cacheConf.setCacheDataOnWrite(true);
+      cacheConf.setEvictOnClose(true);
+      BlockCache cache = cacheConf.getBlockCache();
+
+      // insert data. 2 Rows are added
+      Put put = new Put(ROW);
+      put.addColumn(FAMILY, QUALIFIER, data);
+      table.put(put);
+      put = new Put(ROW1);
+      put.addColumn(FAMILY, QUALIFIER, data);
+      table.put(put);
+      assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
+      // Should create one Hfile with 2 blocks
+      region.flush(true);
+      // read the data and expect same blocks, one new hit, no misses
+      int refCount = 0;
+      // Check how this miss is happening
+      // insert a second column, read the row, no new blocks, 3 new hits
+      byte[] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
+      byte[] data2 = Bytes.add(data, data);
+      put = new Put(ROW);
+      put.addColumn(FAMILY, QUALIFIER2, data2);
+      table.put(put);
+      // flush, one new block
+      System.out.println("Flushing cache");
+      region.flush(true);
+      Iterator<CachedBlock> iterator = cache.iterator();
+      iterateBlockCache(cache, iterator);
+      // Create three sets of scan
+      ScanThread[] scanThreads = initiateScan(table, false);
+      Thread.sleep(100);
+      iterator = cache.iterator();
+      boolean usedBlocksFound = false;
+      while (iterator.hasNext()) {
+        CachedBlock next = iterator.next();
+        BlockCacheKey cacheKey = new BlockCacheKey(next.getFilename(), next.getOffset());
+        if (cache instanceof BucketCache) {
+          refCount = ((BucketCache) cache).getRefCount(cacheKey);
+        } else if (cache instanceof CombinedBlockCache) {
+          refCount = ((CombinedBlockCache) cache).getRefCount(cacheKey);
+        } else {
+          continue;
+        }
+        if (refCount != 0) {
+          // Blocks will be with count 3
+          assertEquals(NO_OF_THREADS, refCount);
+          usedBlocksFound = true;
+        }
+      }
+      // Make a put and do a flush
+      QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
+      data2 = Bytes.add(data, data);
+      put = new Put(ROW1);
+      put.addColumn(FAMILY, QUALIFIER2, data2);
+      table.put(put);
+      // flush, one new block
+      System.out.println("Flushing cache");
+      region.flush(true);
+      assertTrue("Blocks with non zero ref count should be found ", usedBlocksFound);
+      usedBlocksFound = false;
+      System.out.println("Compacting");
+      assertEquals(3, store.getStorefilesCount());
+      store.triggerMajorCompaction();
+      region.compact(true);
+      waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
+      assertEquals(1, store.getStorefilesCount());
+      // Even after compaction is done we will have some blocks that cannot
+      // be evicted this is because the scan is still referencing them
+      iterator = cache.iterator();
+      while (iterator.hasNext()) {
+        CachedBlock next = iterator.next();
+        BlockCacheKey cacheKey = new BlockCacheKey(next.getFilename(), next.getOffset());
+        if (cache instanceof BucketCache) {
+          refCount = ((BucketCache) cache).getRefCount(cacheKey);
+        } else if (cache instanceof CombinedBlockCache) {
+          refCount = ((CombinedBlockCache) cache).getRefCount(cacheKey);
+        } else {
+          continue;
+        }
+        if (refCount != 0) {
+          // Blocks will be with count 3 as they are not yet cleared
+          assertEquals(NO_OF_THREADS, refCount);
+          usedBlocksFound = true;
+        }
+      }
+      assertTrue("Blocks with non zero ref count should be found ", usedBlocksFound);
+      // Should not throw exception
+      compactionLatch.countDown();
+      latch.countDown();
+      for (ScanThread thread : scanThreads) {
+        thread.join();
+      }
+      // by this time all blocks should have been evicted
+      iterator = cache.iterator();
+      // Since a flush and compaction happened after a scan started
+      // we need to ensure that all the original blocks of the compacted file
+      // is also removed.
+      iterateBlockCache(cache, iterator);
+      Result r = table.get(new Get(ROW));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
+      // The gets would be working on new blocks
+      iterator = cache.iterator();
+      iterateBlockCache(cache, iterator);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+
+  @Test
   public void testScanWithException() throws IOException, InterruptedException {
     HTable table = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 5b7e9cc..728029f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -454,9 +454,9 @@ public class TestStoreScanner extends TestCase {
     // normally cause an NPE because scan.store is null.  So as long as we get through these
     // two calls we are good and the bug was quashed.
 
-    scan.updateReaders();
+    scan.updateReaders(new ArrayList<StoreFile>());
 
-    scan.updateReaders();
+    scan.updateReaders(new ArrayList<StoreFile>());
 
     scan.peek();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/555d9b70/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
index ca7b3b1..f598a8d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWideScanner.java
@@ -130,7 +130,7 @@ public class TestWideScanner extends HBaseTestCase {
           ((HRegion.RegionScannerImpl)s).storeHeap.getHeap().iterator();
         while (scanners.hasNext()) {
           StoreScanner ss = (StoreScanner)scanners.next();
-          ss.updateReaders();
+          ss.updateReaders(new ArrayList<StoreFile>());
         }
       } while (more);
 


[20/26] hbase git commit: HBASE-14795 Enhance the spark-hbase scan operations (Zhan Zhang)

Posted by sy...@apache.org.
HBASE-14795 Enhance the spark-hbase scan operations (Zhan Zhang)


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

Branch: refs/heads/hbase-12439
Commit: 676ce01c82c137348e88d0acaa694ad214dc2f12
Parents: f34d3e1
Author: tedyu <yu...@gmail.com>
Authored: Sun Dec 13 18:26:54 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sun Dec 13 18:26:54 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/spark/DefaultSource.scala      | 100 +++-------
 .../hadoop/hbase/spark/HBaseContext.scala       |   2 +-
 .../hadoop/hbase/spark/datasources/Bound.scala  |  91 +++++++++
 .../spark/datasources/HBaseResources.scala      | 147 ++++++++++++++
 .../spark/datasources/HBaseTableScanRDD.scala   | 199 +++++++++++++++++++
 .../datasources/SerializableConfiguration.scala |  46 +++++
 .../hbase/spark/datasources/package.scala       |  36 ++++
 7 files changed, 545 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
index 23480bb..664cf57 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.spark
 import java.util
 import java.util.concurrent.ConcurrentLinkedQueue
 
-import org.apache.hadoop.hbase.client.{ConnectionFactory, Get, Result, Scan}
+import org.apache.hadoop.hbase.client._
+import org.apache.hadoop.hbase.spark.datasources.{HBaseTableScanRDD, HBaseRegion, SerializableConfiguration}
 import org.apache.hadoop.hbase.types._
-import org.apache.hadoop.hbase.util.{SimplePositionedMutableByteRange,
-PositionedByteRange, Bytes}
-import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
+import org.apache.hadoop.hbase.util.{Bytes, PositionedByteRange, SimplePositionedMutableByteRange}
+import org.apache.hadoop.hbase.{HBaseConfiguration, TableName}
 import org.apache.spark.Logging
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.types.DataType
@@ -159,7 +159,7 @@ class DefaultSource extends RelationProvider with Logging {
  *                                connection information
  * @param sqlContext              SparkSQL context
  */
-class HBaseRelation (val tableName:String,
+case class HBaseRelation (val tableName:String,
                      val schemaMappingDefinition:
                      java.util.HashMap[String, SchemaQualifierDefinition],
                      val batchingNum:Int,
@@ -179,6 +179,9 @@ class HBaseRelation (val tableName:String,
     new HBaseContext(sqlContext.sparkContext, config)
   }
 
+  val wrappedConf = new SerializableConfiguration(hbaseContext.config)
+  def hbaseConf = wrappedConf.value
+
   /**
    * Generates a Spark SQL schema object so Spark SQL knows what is being
    * provided by this BaseRelation
@@ -222,6 +225,7 @@ class HBaseRelation (val tableName:String,
    */
   override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
 
+
     val pushDownTuple = buildPushDownPredicatesResource(filters)
     val pushDownRowKeyFilter = pushDownTuple._1
     var pushDownDynamicLogicExpression = pushDownTuple._2
@@ -253,7 +257,6 @@ class HBaseRelation (val tableName:String,
     //retain the information for unit testing checks
     DefaultSourceStaticUtils.populateLatestExecutionRules(pushDownRowKeyFilter,
       pushDownDynamicLogicExpression)
-    var resultRDD: RDD[Row] = null
 
     val getList = new util.ArrayList[Get]()
     val rddList = new util.ArrayList[RDD[Row]]()
@@ -268,77 +271,24 @@ class HBaseRelation (val tableName:String,
       getList.add(get)
     })
 
-    val rangeIt = pushDownRowKeyFilter.ranges.iterator
-
-    while (rangeIt.hasNext) {
-      val r = rangeIt.next()
-
-      val scan = new Scan()
-      scan.setBatch(batchingNum)
-      scan.setCaching(cachingNum)
-      requiredQualifierDefinitionList.foreach( d =>
-        if (d.columnFamilyBytes.length > 0)
-          scan.addColumn(d.columnFamilyBytes, d.qualifierBytes))
-
-      if (usePushDownColumnFilter && pushDownDynamicLogicExpression != null) {
-        val pushDownFilterJava =
-          new SparkSQLPushDownFilter(pushDownDynamicLogicExpression,
-            valueArray, requiredQualifierDefinitionList)
-
-        scan.setFilter(pushDownFilterJava)
-      }
-
-      //Check if there is a lower bound
-      if (r.lowerBound != null && r.lowerBound.length > 0) {
-
-        if (r.isLowerBoundEqualTo) {
-          //HBase startRow is inclusive: Therefore it acts like  isLowerBoundEqualTo
-          // by default
-          scan.setStartRow(r.lowerBound)
-        } else {
-          //Since we don't equalTo we want the next value we need
-          // to add another byte to the start key.  That new byte will be
-          // the min byte value.
-          val newArray = new Array[Byte](r.lowerBound.length + 1)
-          System.arraycopy(r.lowerBound, 0, newArray, 0, r.lowerBound.length)
-
-          //new Min Byte
-          newArray(r.lowerBound.length) = Byte.MinValue
-          scan.setStartRow(newArray)
-        }
+    val pushDownFilterJava = if (usePushDownColumnFilter && pushDownDynamicLogicExpression != null) {
+        Some(new SparkSQLPushDownFilter(pushDownDynamicLogicExpression,
+          valueArray, requiredQualifierDefinitionList))
+    } else {
+      None
+    }
+    val hRdd = new HBaseTableScanRDD(this, pushDownFilterJava, requiredQualifierDefinitionList.seq)
+    pushDownRowKeyFilter.ranges.foreach(hRdd.addRange(_))
+    var resultRDD: RDD[Row] = {
+      val tmp = hRdd.map{ r =>
+        Row.fromSeq(requiredColumns.map(c =>
+          DefaultSourceStaticUtils.getValue(c, serializableDefinitionMap, r)))
       }
-
-      //Check if there is a upperBound
-      if (r.upperBound != null && r.upperBound.length > 0) {
-        if (r.isUpperBoundEqualTo) {
-          //HBase stopRow is exclusive: therefore it DOESN'T ast like isUpperBoundEqualTo
-          // by default.  So we need to add a new max byte to the stopRow key
-          val newArray = new Array[Byte](r.upperBound.length + 1)
-          System.arraycopy(r.upperBound, 0, newArray, 0, r.upperBound.length)
-
-          //New Max Bytes
-          newArray(r.upperBound.length) = Byte.MaxValue
-          scan.setStopRow(newArray)
-        } else {
-          //Here equalTo is false for Upper bound which is exclusive and
-          // HBase stopRow acts like that by default so no need to mutate the
-          // rowKey
-          scan.setStopRow(r.upperBound)
-        }
+      if (tmp.partitions.size > 0) {
+        tmp
+      } else {
+        null
       }
-
-      val rdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan).map(r => {
-        Row.fromSeq(requiredColumns.map(c =>
-          DefaultSourceStaticUtils.getValue(c, serializableDefinitionMap, r._2)))
-      })
-      rddList.add(rdd)
-    }
-
-    //If there is more then one RDD then we have to union them together
-    for (i <- 0 until rddList.size()) {
-      if (resultRDD == null) resultRDD = rddList.get(i)
-      else resultRDD = resultRDD.union(rddList.get(i))
-
     }
 
     //If there are gets then we can get them from the driver and union that rdd in

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
index f8c24f2..2d21e69 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
@@ -57,7 +57,7 @@ import scala.collection.mutable
   * to the working and managing the life cycle of HConnections.
  */
 class HBaseContext(@transient sc: SparkContext,
-                   @transient config: Configuration,
+                   @transient val config: Configuration,
                    val tmpHdfsConfgFile: String = null)
   extends Serializable with Logging {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
new file mode 100644
index 0000000..c869f31
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
@@ -0,0 +1,91 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.spark.datasources
+
+import org.apache.hadoop.hbase.spark.SparkSQLPushDownFilter
+import org.apache.spark.Partition
+import org.apache.hadoop.hbase.spark.hbase._
+
+/**
+ * The Bound represent the boudary for the scan
+ *
+ * @param b The byte array of the bound
+ * @param inc inclusive or not.
+ */
+case class Bound(b: Array[Byte], inc: Boolean)
+// The non-overlapping ranges we need to scan, if lower is equal to upper, it is a get request
+case class Range(lower: Option[Bound], upper: Option[Bound])
+object Range {
+  def apply(region: HBaseRegion): Range = {
+    Range(region.start.map(Bound(_, true)), if (region.end.get.size == 0) {
+      None
+    } else {
+      region.end.map((Bound(_, false)))
+    })
+  }
+}
+
+object Ranges {
+  // We assume that
+  // 1. r.lower.inc is true, and r.upper.inc is false
+  // 2. for each range in rs, its upper.inc is false
+  def and(r: Range, rs: Seq[Range]): Seq[Range] = {
+    rs.flatMap{ s =>
+      val lower = s.lower.map { x =>
+        // the scan has lower bound
+        r.lower.map { y =>
+          // the region has lower bound
+          if (ord.compare(x.b, y.b) < 0) {
+            // scan lower bound is smaller than region server lower bound
+            Some(y)
+          } else {
+            // scan low bound is greater or equal to region server lower bound
+            Some(x)
+          }
+        }.getOrElse(Some(x))
+      }.getOrElse(r.lower)
+
+      val upper =  s.upper.map { x =>
+        // the scan has upper bound
+        r.upper.map { y =>
+          // the region has upper bound
+          if (ord.compare(x.b, y.b) >= 0) {
+            // scan upper bound is larger than server upper bound
+            // but region server scan stop is exclusive. It is OK here.
+            Some(y)
+          } else {
+            // scan upper bound is less or equal to region server upper bound
+            Some(x)
+          }
+        }.getOrElse(Some(x))
+      }.getOrElse(r.upper)
+
+      val c = lower.map { case x =>
+        upper.map { case y =>
+          ord.compare(x.b, y.b)
+        }.getOrElse(-1)
+      }.getOrElse(-1)
+      if (c < 0) {
+        Some(Range(lower, upper))
+      } else {
+        None
+      }
+    }.seq
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
new file mode 100644
index 0000000..4417520
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
@@ -0,0 +1,147 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.spark.datasources
+
+import org.apache.hadoop.hbase.TableName
+import org.apache.hadoop.hbase.client._
+import org.apache.hadoop.hbase.spark.HBaseRelation
+
+// Resource and ReferencedResources are defined for extensibility,
+// e.g., consolidate scan and bulkGet in the future work.
+
+// User has to invoke release explicitly to release the resource,
+// and potentially parent resources
+trait Resource {
+  def release(): Unit
+}
+
+case class ScanResource(tbr: TableResource, rs: ResultScanner) extends Resource {
+  def release() {
+    rs.close()
+    tbr.release()
+  }
+}
+
+trait ReferencedResource {
+  var count: Int = 0
+  def init(): Unit
+  def destroy(): Unit
+  def acquire() = synchronized {
+    try {
+      count += 1
+      if (count == 1) {
+        init()
+      }
+    } catch {
+      case e: Throwable =>
+        release()
+        throw e
+    }
+  }
+
+  def release() = synchronized {
+    count -= 1
+    if (count == 0) {
+      destroy()
+    }
+  }
+
+  def releaseOnException[T](func: => T): T = {
+    acquire()
+    val ret = {
+      try {
+        func
+      } catch {
+        case e: Throwable =>
+          release()
+          throw e
+      }
+    }
+    ret
+  }
+}
+
+case class TableResource(relation: HBaseRelation) extends ReferencedResource {
+  var connection: Connection = _
+  var table: Table = _
+
+  override def init(): Unit = {
+    connection = ConnectionFactory.createConnection(relation.hbaseConf)
+    table = connection.getTable(TableName.valueOf(relation.tableName))
+  }
+
+  override def destroy(): Unit = {
+    if (table != null) {
+      table.close()
+      table = null
+    }
+    if (connection != null) {
+      connection.close()
+      connection = null
+    }
+  }
+
+  def getScanner(scan: Scan): ScanResource = releaseOnException {
+    ScanResource(this, table.getScanner(scan))
+  }
+}
+
+case class RegionResource(relation: HBaseRelation) extends ReferencedResource {
+  var connection: Connection = _
+  var rl: RegionLocator = _
+  val regions = releaseOnException {
+    val keys = rl.getStartEndKeys
+    keys.getFirst.zip(keys.getSecond)
+      .zipWithIndex
+      .map(x =>
+      HBaseRegion(x._2,
+        Some(x._1._1),
+        Some(x._1._2),
+        Some(rl.getRegionLocation(x._1._1).getHostname)))
+  }
+
+  override def init(): Unit = {
+    connection = ConnectionFactory.createConnection(relation.hbaseConf)
+    rl = connection.getRegionLocator(TableName.valueOf(relation.tableName))
+  }
+
+  override def destroy(): Unit = {
+    if (rl != null) {
+      rl.close()
+      rl = null
+    }
+    if (connection != null) {
+      connection.close()
+      connection = null
+    }
+  }
+}
+
+object HBaseResources{
+  implicit def ScanResToScan(sr: ScanResource): ResultScanner = {
+    sr.rs
+  }
+
+  implicit def TableResToTable(tr: TableResource): Table = {
+    tr.table
+  }
+
+  implicit def RegionResToRegions(rr: RegionResource): Seq[HBaseRegion] = {
+    rr.regions
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
new file mode 100644
index 0000000..958a986
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
@@ -0,0 +1,199 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.spark.datasources
+
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.hadoop.hbase.TableName
+import org.apache.hadoop.hbase.client._
+import org.apache.hadoop.hbase.filter.Filter
+import org.apache.hadoop.hbase.spark.{ScanRange, SchemaQualifierDefinition, HBaseRelation, SparkSQLPushDownFilter}
+import org.apache.hadoop.hbase.spark.hbase._
+import org.apache.hadoop.hbase.spark.datasources.HBaseResources._
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.{TaskContext, Logging, Partition}
+import org.apache.spark.rdd.RDD
+
+import scala.collection.mutable
+
+
+class HBaseTableScanRDD(relation: HBaseRelation,
+     @transient val filter: Option[SparkSQLPushDownFilter] = None,
+     val columns: Seq[SchemaQualifierDefinition] = Seq.empty
+     )extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging  {
+  var ranges = Seq.empty[Range]
+  def addRange(r: ScanRange) = {
+    val lower = if (r.lowerBound != null && r.lowerBound.length > 0) {
+      Some(Bound(r.lowerBound, r.isLowerBoundEqualTo))
+    } else {
+      None
+    }
+    val upper = if (r.upperBound != null && r.upperBound.length > 0) {
+      if (!r.isUpperBoundEqualTo) {
+        Some(Bound(r.upperBound, false))
+      } else {
+
+        // HBase stopRow is exclusive: therefore it DOESN'T act like isUpperBoundEqualTo
+        // by default.  So we need to add a new max byte to the stopRow key
+        val newArray = new Array[Byte](r.upperBound.length + 1)
+        System.arraycopy(r.upperBound, 0, newArray, 0, r.upperBound.length)
+
+        //New Max Bytes
+        newArray(r.upperBound.length) = ByteMin
+        Some(Bound(newArray, false))
+      }
+    } else {
+      None
+    }
+    ranges :+= Range(lower, upper)
+  }
+
+  override def getPartitions: Array[Partition] = {
+    val regions = RegionResource(relation)
+    var idx = 0
+    logDebug(s"There are ${regions.size} regions")
+    val ps = regions.flatMap { x =>
+      val rs = Ranges.and(Range(x), ranges)
+      if (rs.size > 0) {
+        if(log.isDebugEnabled) {
+          rs.foreach(x => logDebug(x.toString))
+        }
+        idx += 1
+        Some(HBaseScanPartition(idx - 1, x, rs, SerializedFilter.toSerializedTypedFilter(filter)))
+      } else {
+        None
+      }
+    }.toArray
+    regions.release()
+    ps.asInstanceOf[Array[Partition]]
+  }
+
+
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    split.asInstanceOf[HBaseScanPartition].regions.server.map {
+      identity
+    }.toSeq
+  }
+
+  private def buildScan(range: Range,
+      filter: Option[SparkSQLPushDownFilter],
+      columns: Seq[SchemaQualifierDefinition]): Scan = {
+    val scan = (range.lower, range.upper) match {
+      case (Some(Bound(a, b)), Some(Bound(c, d))) => new Scan(a, c)
+      case (None, Some(Bound(c, d))) => new Scan(Array[Byte](), c)
+      case (Some(Bound(a, b)), None) => new Scan(a)
+      case (None, None) => new Scan()
+    }
+
+    columns.foreach { d =>
+      if (d.columnFamilyBytes.length > 0) {
+        scan.addColumn(d.columnFamilyBytes, d.qualifierBytes)
+      }
+    }
+    scan.setBatch(relation.batchingNum)
+    scan.setCaching(relation.cachingNum)
+    filter.foreach(scan.setFilter(_))
+    scan
+  }
+  private def toResultIterator(scanner: ScanResource): Iterator[Result] = {
+    val iterator = new Iterator[Result] {
+      var cur: Option[Result] = None
+      override def hasNext: Boolean = {
+        if (cur.isEmpty) {
+          val r = scanner.next()
+          if (r == null) {
+            rddResources.release(scanner)
+          } else {
+            cur = Some(r)
+          }
+        }
+        cur.isDefined
+      }
+      override def next(): Result = {
+        hasNext
+        val ret = cur.get
+        cur = None
+        ret
+      }
+    }
+    iterator
+  }
+  lazy val rddResources = RDDResources(new mutable.HashSet[Resource]())
+
+  private def close() {
+    rddResources.release()
+  }
+
+  override def compute(split: Partition, context: TaskContext): Iterator[Result] = {
+    val partition = split.asInstanceOf[HBaseScanPartition]
+
+    val scans = partition.scanRanges
+      .map(buildScan(_, SerializedFilter.fromSerializedFilter(partition.sf), columns))
+    val tableResource = TableResource(relation)
+    context.addTaskCompletionListener(context => close())
+    val sIts = scans.par
+      .map(tableResource.getScanner(_))
+      .map(toResultIterator(_))
+      .fold(Iterator.empty: Iterator[Result]){ case (x, y) =>
+      x ++ y
+    }
+    sIts
+  }
+}
+
+case class SerializedFilter(b: Option[Array[Byte]])
+
+object SerializedFilter {
+  def toSerializedTypedFilter(f: Option[SparkSQLPushDownFilter]): SerializedFilter = {
+    SerializedFilter(f.map(_.toByteArray))
+  }
+
+  def fromSerializedFilter(sf: SerializedFilter): Option[SparkSQLPushDownFilter] = {
+    sf.b.map(SparkSQLPushDownFilter.parseFrom(_))
+  }
+}
+
+private[hbase] case class HBaseRegion(
+    override val index: Int,
+    val start: Option[HBaseType] = None,
+    val end: Option[HBaseType] = None,
+    val server: Option[String] = None) extends Partition
+
+
+private[hbase] case class HBaseScanPartition(
+    override val index: Int,
+    val regions: HBaseRegion,
+    val scanRanges: Seq[Range],
+    val sf: SerializedFilter) extends Partition
+
+case class RDDResources(set: mutable.HashSet[Resource]) {
+  def addResource(s: Resource) {
+    set += s
+  }
+  def release() {
+    set.foreach(release(_))
+  }
+  def release(rs: Resource) {
+    try {
+      rs.release()
+    } finally {
+      set.remove(rs)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
new file mode 100644
index 0000000..2452a52
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.spark.datasources
+
+import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.util.Utils
+
+import scala.util.control.NonFatal
+
+class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
+  private def writeObject(out: ObjectOutputStream): Unit = tryOrIOException {
+    out.defaultWriteObject()
+    value.write(out)
+  }
+
+  private def readObject(in: ObjectInputStream): Unit = tryOrIOException {
+    value = new Configuration(false)
+    value.readFields(in)
+  }
+
+  def tryOrIOException(block: => Unit) {
+    try {
+      block
+    } catch {
+      case e: IOException => throw e
+      case NonFatal(t) => throw new IOException(t)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/676ce01c/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala
new file mode 100644
index 0000000..4ff0413
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala
@@ -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.
+ */
+
+package org.apache.hadoop.hbase.spark
+
+import org.apache.hadoop.hbase.util.Bytes
+
+import scala.math.Ordering
+
+package object hbase {
+  type HBaseType = Array[Byte]
+  val ByteMax = -1.asInstanceOf[Byte]
+  val ByteMin = 0.asInstanceOf[Byte]
+  val ord: Ordering[HBaseType] = new Ordering[HBaseType] {
+    def compare(x: Array[Byte], y: Array[Byte]): Int = {
+      return Bytes.compareTo(x, y)
+    }
+  }
+  //Do not use BinaryType.ordering
+  implicit val order: Ordering[HBaseType] = ord
+
+}


[23/26] hbase git commit: HBASE-14929 There is a space missing from table "foo" is not currently available (Carlos A. Morillo)

Posted by sy...@apache.org.
HBASE-14929 There is a space missing from table "foo" is not currently available (Carlos A. Morillo)


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

Branch: refs/heads/hbase-12439
Commit: 10d86c7daea1f27cb7f9849c907696431cc6ab5c
Parents: 0462225
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Mon Dec 14 16:07:05 2015 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Mon Dec 14 16:07:05 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/10d86c7d/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 369ae90..5064c05 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -321,7 +321,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       RegionLocator regionLocator) throws TableNotFoundException, IOException  {
 
     if (!admin.isTableAvailable(regionLocator.getName())) {
-      throw new TableNotFoundException("Table " + table.getName() + "is not currently available.");
+      throw new TableNotFoundException("Table " + table.getName() + " is not currently available.");
     }
 
     ExecutorService pool = createExecutorService();


[24/26] hbase git commit: HBASE-14895 Seek only to the newly flushed file on scanner reset on flush - Addendum (Ram)

Posted by sy...@apache.org.
HBASE-14895 Seek only to the newly flushed file on scanner reset on flush
- Addendum (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 1822bb5d9578e3e650115a220132ab85e7db732a
Parents: 10d86c7
Author: ramkrishna <ra...@gmail.com>
Authored: Tue Dec 15 10:34:25 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Tue Dec 15 10:34:25 2015 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/regionserver/StoreScanner.java   | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1822bb5d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 987a3f5..5fdfa79 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -414,7 +414,6 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   @Override
   public Cell peek() {
-    checkFlushed();
     if (this.heap == null) {
       return this.lastTop;
     }
@@ -810,7 +809,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       // If there is a flush and the current scan is notified on the flush ensure that the 
       // scan's heap gets reset and we do a seek on the newly flushed file.
       if(!this.closing) {
-        this.lastTop = this.heap.peek();
+        this.lastTop = this.peek();
       } else {
         return false;
       }


[25/26] hbase git commit: HBASE-14979 Update to the newest Zookeeper release

Posted by sy...@apache.org.
HBASE-14979 Update to the newest Zookeeper release


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

Branch: refs/heads/hbase-12439
Commit: ef92a6a067230cae96d3e3267da5a18ac5ad89c6
Parents: 1822bb5
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Dec 14 18:35:40 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Mon Dec 14 21:07:20 2015 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ef92a6a0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3f21cd4..0999120 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1199,7 +1199,7 @@
     <protobuf.version>2.5.0</protobuf.version>
     <thrift.path>thrift</thrift.path>
     <thrift.version>0.9.3</thrift.version>
-    <zookeeper.version>3.4.6</zookeeper.version>
+    <zookeeper.version>3.4.7</zookeeper.version>
     <slf4j.version>1.7.7</slf4j.version>
     <clover.version>4.0.3</clover.version>
     <jamon-runtime.version>2.4.1</jamon-runtime.version>


[02/26] hbase git commit: HBASE-14942 Allow turning off BoundedByteBufferPool

Posted by sy...@apache.org.
HBASE-14942 Allow turning off BoundedByteBufferPool


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

Branch: refs/heads/hbase-12439
Commit: ba3aa9a9b187322bfdb64e676778872ad69552aa
Parents: 0e147a9
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Dec 7 11:01:51 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Dec 9 15:29:25 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java    | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ba3aa9a9/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 402bca0..0db7383 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -2004,13 +2004,17 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       final InetSocketAddress bindAddress, Configuration conf,
       RpcScheduler scheduler)
       throws IOException {
-    this.reservoir = new BoundedByteBufferPool(
-      conf.getInt("hbase.ipc.server.reservoir.max.buffer.size",  1024 * 1024),
-      conf.getInt("hbase.ipc.server.reservoir.initial.buffer.size", 16 * 1024),
-      // Make the max twice the number of handlers to be safe.
-      conf.getInt("hbase.ipc.server.reservoir.initial.max",
-        conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
-          HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT) * 2));
+    if (conf.getBoolean("hbase.ipc.server.reservoir.enabled", true)) {
+      this.reservoir = new BoundedByteBufferPool(
+          conf.getInt("hbase.ipc.server.reservoir.max.buffer.size", 1024 * 1024),
+          conf.getInt("hbase.ipc.server.reservoir.initial.buffer.size", 16 * 1024),
+          // Make the max twice the number of handlers to be safe.
+          conf.getInt("hbase.ipc.server.reservoir.initial.max",
+              conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
+                  HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT) * 2));
+    } else {
+      reservoir = null;
+    }
     this.server = server;
     this.services = services;
     this.bindAddress = bindAddress;


[15/26] hbase git commit: HBASE-14745 Shade the last few dependencies in hbase-shaded-client

Posted by sy...@apache.org.
HBASE-14745 Shade the last few dependencies in hbase-shaded-client


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

Branch: refs/heads/hbase-12439
Commit: abb2e95f66191588971c6bba800f6b0dcbd7ad37
Parents: 48e217a
Author: Elliott Clark <ec...@apache.org>
Authored: Wed Nov 4 13:40:46 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Dec 10 18:44:12 2015 -0800

----------------------------------------------------------------------
 hbase-shaded/pom.xml | 28 ++++++++++++++++++++++++++++
 pom.xml              |  2 ++
 2 files changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/abb2e95f/hbase-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 78b8270..4e38c88 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -294,12 +294,40 @@
                                         <pattern>org.apache.commons.el</pattern>
                                         <shadedPattern>org.apache.hadoop.hbase.shaded.org.apache.commons.el</shadedPattern>
                                     </relocation>
+                                    <relocation>
+                                        <pattern>org.apache.commons.httpclient</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.org.apache.commons.httpclient</shadedPattern>
+                                    </relocation>
+                                    <relocation>
+                                        <pattern>org.apache.commons.compress</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.org.apache.commons.compress</shadedPattern>
+                                    </relocation>
+                                    <relocation>
+                                        <pattern>org.apache.commons.digester</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.org.apache.commons.digester</shadedPattern>
+                                    </relocation>
+                                    <relocation>
+                                        <pattern>org.apache.commons.codec</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.org.apache.commons.codec</shadedPattern>
+                                    </relocation>
 
                                     <!-- top level net-->
                                     <relocation>
                                         <pattern>net.iharder</pattern>
                                         <shadedPattern>org.apache.hadoop.hbase.shaded.net.iharder</shadedPattern>
                                     </relocation>
+
+                                    <!-- junit -->
+                                    <relocation>
+                                        <pattern>junit</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.junit</shadedPattern>
+                                    </relocation>
+                                    <relocation>
+                                        <pattern>org.junit</pattern>
+                                        <shadedPattern>org.apache.hadoop.hbase.shaded.org.junit</shadedPattern>
+                                    </relocation>
+
+
                                 </relocations>
                                 <transformers>
                                   <!-- Need to filter out some extraneous license files.

http://git-wip-us.apache.org/repos/asf/hbase/blob/abb2e95f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ee5784..3f21cd4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -822,6 +822,8 @@
               <exclude>**/.settings/**</exclude>
               <exclude>**/patchprocess/**</exclude>
               <exclude>src/main/site/resources/repo/**</exclude>
+              <exclude>**/dependency-reduced-pom.xml</exclude>
+              <exclude>**/rat.txt</exclude>
             </excludes>
           </configuration>
         </plugin>


[22/26] hbase git commit: HBASE-14936 CombinedBlockCache should overwrite CacheStats#rollMetricsPeriod() (Jianwei Cui)

Posted by sy...@apache.org.
HBASE-14936 CombinedBlockCache should overwrite CacheStats#rollMetricsPeriod() (Jianwei Cui)


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

Branch: refs/heads/hbase-12439
Commit: 04622254f7209c5cfeadcfa137a97fbed161075a
Parents: 555d9b7
Author: chenheng <ch...@apache.org>
Authored: Mon Dec 14 17:20:50 2015 +0800
Committer: chenheng <ch...@apache.org>
Committed: Mon Dec 14 17:20:50 2015 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/CacheStats.java       |  7 +-
 .../hbase/io/hfile/CombinedBlockCache.java      | 48 ++++++----
 .../hbase/io/hfile/TestCombinedBlockCache.java  | 98 ++++++++++++++++++++
 3 files changed, 134 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/04622254/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
index 2dae66f..fff6585 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
@@ -260,13 +260,14 @@ public class CacheStats {
   }
 
   public double getHitRatioPastNPeriods() {
-    double ratio = ((double)sum(hitCounts)/(double)sum(requestCounts));
+    double ratio = ((double)getSumHitCountsPastNPeriods() /
+        (double)getSumRequestCountsPastNPeriods());
     return Double.isNaN(ratio) ? 0 : ratio;
   }
 
   public double getHitCachingRatioPastNPeriods() {
-    double ratio =
-      ((double)sum(hitCachingCounts)/(double)sum(requestCachingCounts));
+    double ratio = ((double)getSumHitCachingCountsPastNPeriods() /
+        (double)getSumRequestCachingCountsPastNPeriods());
     return Double.isNaN(ratio) ? 0 : ratio;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04622254/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 3f65823..22bffee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -203,22 +203,38 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
     }
 
     @Override
-    public double getHitRatioPastNPeriods() {
-      double ratio = ((double) (lruCacheStats.getSumHitCountsPastNPeriods() + bucketCacheStats
-          .getSumHitCountsPastNPeriods()) / (double) (lruCacheStats
-          .getSumRequestCountsPastNPeriods() + bucketCacheStats
-          .getSumRequestCountsPastNPeriods()));
-      return Double.isNaN(ratio) ? 0 : ratio;
-    }
-
-    @Override
-    public double getHitCachingRatioPastNPeriods() {
-      double ratio = ((double) (lruCacheStats
-          .getSumHitCachingCountsPastNPeriods() + bucketCacheStats
-          .getSumHitCachingCountsPastNPeriods()) / (double) (lruCacheStats
-          .getSumRequestCachingCountsPastNPeriods() + bucketCacheStats
-          .getSumRequestCachingCountsPastNPeriods()));
-      return Double.isNaN(ratio) ? 0 : ratio;
+    public void rollMetricsPeriod() {
+      lruCacheStats.rollMetricsPeriod();
+      bucketCacheStats.rollMetricsPeriod();
+    }
+    
+    @Override
+    public long getFailedInserts() {
+      return lruCacheStats.getFailedInserts() + bucketCacheStats.getFailedInserts();
+    }
+
+    @Override
+    public long getSumHitCountsPastNPeriods() {
+      return lruCacheStats.getSumHitCountsPastNPeriods()
+          + bucketCacheStats.getSumHitCountsPastNPeriods();
+    }
+    
+    @Override
+    public long getSumRequestCountsPastNPeriods() {
+      return lruCacheStats.getSumRequestCountsPastNPeriods()
+          + bucketCacheStats.getSumRequestCountsPastNPeriods();
+    }
+    
+    @Override
+    public long getSumHitCachingCountsPastNPeriods() {
+      return lruCacheStats.getSumHitCachingCountsPastNPeriods()
+          + bucketCacheStats.getSumHitCachingCountsPastNPeriods();
+    }
+
+    @Override
+    public long getSumRequestCachingCountsPastNPeriods() {
+      return lruCacheStats.getSumRequestCachingCountsPastNPeriods()
+          + bucketCacheStats.getSumRequestCachingCountsPastNPeriods();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04622254/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java
new file mode 100644
index 0000000..50bf331
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCombinedBlockCache.java
@@ -0,0 +1,98 @@
+/**
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import static org.junit.Assert.assertEquals;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache.CombinedCacheStats;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class})
+public class TestCombinedBlockCache {
+  @Test
+  public void testCombinedCacheStats() {
+    CacheStats lruCacheStats = new CacheStats("lruCacheStats", 2);
+    CacheStats bucketCacheStats = new CacheStats("bucketCacheStats", 2);
+    CombinedCacheStats stats =
+        new CombinedCacheStats(lruCacheStats, bucketCacheStats);
+    
+    double delta = 0.01;
+    
+    // period 1:
+    // lru cache: 1 hit caching, 1 miss caching
+    // bucket cache: 2 hit non-caching,1 miss non-caching/primary,1 fail insert
+    lruCacheStats.hit(true);
+    lruCacheStats.miss(true, false);
+    bucketCacheStats.hit(false);
+    bucketCacheStats.hit(false);
+    bucketCacheStats.miss(false, true);
+    
+    assertEquals(5, stats.getRequestCount());
+    assertEquals(2, stats.getRequestCachingCount());
+    assertEquals(2, stats.getMissCount());
+    assertEquals(1, stats.getPrimaryMissCount());
+    assertEquals(1, stats.getMissCachingCount());
+    assertEquals(3, stats.getHitCount());
+    assertEquals(3, stats.getPrimaryHitCount());
+    assertEquals(1, stats.getHitCachingCount());
+    assertEquals(0.6, stats.getHitRatio(), delta);
+    assertEquals(0.5, stats.getHitCachingRatio(), delta);
+    assertEquals(0.4, stats.getMissRatio(), delta);
+    assertEquals(0.5, stats.getMissCachingRatio(), delta);
+    
+    
+    // lru cache: 2 evicted, 1 evict
+    // bucket cache: 1 evict
+    lruCacheStats.evicted(1000, true);
+    lruCacheStats.evicted(1000, false);
+    lruCacheStats.evict();
+    bucketCacheStats.evict();
+    assertEquals(2, stats.getEvictionCount());
+    assertEquals(2, stats.getEvictedCount());
+    assertEquals(1, stats.getPrimaryEvictedCount());
+    assertEquals(1.0, stats.evictedPerEviction(), delta);
+    
+    // lru cache:  1 fail insert
+    lruCacheStats.failInsert();
+    assertEquals(1, stats.getFailedInserts());
+    
+    // rollMetricsPeriod
+    stats.rollMetricsPeriod();
+    assertEquals(3, stats.getSumHitCountsPastNPeriods());
+    assertEquals(5, stats.getSumRequestCountsPastNPeriods());
+    assertEquals(1, stats.getSumHitCachingCountsPastNPeriods());
+    assertEquals(2, stats.getSumRequestCachingCountsPastNPeriods());
+    assertEquals(0.6, stats.getHitRatioPastNPeriods(), delta);
+    assertEquals(0.5, stats.getHitCachingRatioPastNPeriods(), delta);
+    
+    // period 2:
+    // lru cache: 3 hit caching
+    lruCacheStats.hit(true);
+    lruCacheStats.hit(true);
+    lruCacheStats.hit(true);
+    stats.rollMetricsPeriod();
+    assertEquals(6, stats.getSumHitCountsPastNPeriods());
+    assertEquals(8, stats.getSumRequestCountsPastNPeriods());
+    assertEquals(4, stats.getSumHitCachingCountsPastNPeriods());
+    assertEquals(5, stats.getSumRequestCachingCountsPastNPeriods());
+    assertEquals(0.75, stats.getHitRatioPastNPeriods(), delta);
+    assertEquals(0.8, stats.getHitCachingRatioPastNPeriods(), delta);
+  }
+}


[10/26] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index 82a599c..13a5936 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -121,7 +121,7 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
       if(Arrays.equals(columnDescriptor.getName(), mobColumnFamily)) {
         columnDescriptor.setMobEnabled(true);
         columnDescriptor.setMobThreshold((long) threshold);
-        admin.modifyColumn(tableName, columnDescriptor);
+        admin.modifyColumnFamily(tableName, columnDescriptor);
       }
     }
     LOG.info("Enabling table " + getTablename());

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 006c3e7..71a3344 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -2682,13 +2682,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
 
   /**
    * Returns a Admin instance.
-   * This instance is shared between HBaseTestingUtility instance users.
-   * Closing it has no effect, it will be closed automatically when the
-   * cluster shutdowns
+   * This instance is shared between HBaseTestingUtility instance users. Closing it has no effect,
+   * it will be closed automatically when the cluster shutdowns
    *
-   * @return An Admin instance.
-   * @throws IOException
+   * @return HBaseAdmin instance which is guaranteed to support only {@link Admin} interface.
+   *   Functions in HBaseAdmin not provided by {@link Admin} interface can be changed/deleted
+   *   anytime.
+   * @deprecated Since 2.0. Will be removed in 3.0. Use {@link #getAdmin()} instead.
    */
+  @Deprecated
   public synchronized HBaseAdmin getHBaseAdmin()
   throws IOException {
     if (hbaseAdmin == null){
@@ -2697,8 +2699,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return hbaseAdmin;
   }
 
-  private HBaseAdmin hbaseAdmin = null;
+  /**
+   * Returns an Admin instance which is shared between HBaseTestingUtility instance users.
+   * Closing it has no effect, it will be closed automatically when the cluster shutdowns
+   */
+  public synchronized Admin getAdmin() throws IOException {
+    if (hbaseAdmin == null){
+      this.hbaseAdmin = (HBaseAdmin) getConnection().getAdmin();
+    }
+    return hbaseAdmin;
+  }
 
+  private HBaseAdmin hbaseAdmin = null;
 
   /**
    * Returns a ZooKeeperWatcher instance.

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
index 5450148..989192d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java
@@ -88,7 +88,7 @@ public class TestAcidGuarantees implements Tool {
       // force mob enabled such that all data is mob data
       hcd.setMobEnabled(true);
       hcd.setMobThreshold(4);
-      util.getHBaseAdmin().modifyColumn(TABLE_NAME, hcd);
+      util.getHBaseAdmin().modifyColumnFamily(TABLE_NAME, hcd);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 0f286b6..e510d28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -465,8 +465,8 @@ public class TestAdmin2 {
         onlineRegions.contains(info));
   }
 
-  private Admin createTable(TableName tableName) throws IOException {
-    Admin admin = TEST_UTIL.getHBaseAdmin();
+  private HBaseAdmin createTable(TableName tableName) throws IOException {
+    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
 
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor("value");

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 7278892..5e302d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -400,8 +400,8 @@ public class TestMetaWithReplicas {
       }
     }
     assert(moveToServer != null);
-    String tableName = "randomTable5678";
-    TEST_UTIL.createTable(TableName.valueOf(tableName), "f");
+    TableName tableName = TableName.valueOf("randomTable5678");
+    TEST_UTIL.createTable(tableName, "f");
     assertTrue(TEST_UTIL.getHBaseAdmin().tableExists(tableName));
     TEST_UTIL.getHBaseAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
         Bytes.toBytes(moveToServer.getServerName()));
@@ -413,8 +413,8 @@ public class TestMetaWithReplicas {
       i++;
     } while (!moveToServer.equals(currentServer) && i < 1000); //wait for 10 seconds overall
     assert(i != 1000);
-    TEST_UTIL.getHBaseAdmin().disableTable("randomTable5678");
-    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled("randomTable5678"));
+    TEST_UTIL.getHBaseAdmin().disableTable(tableName);
+    assertTrue(TEST_UTIL.getHBaseAdmin().isTableDisabled(tableName));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
index fff6f44..267201f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestExpiredMobFileCleaner.java
@@ -100,7 +100,7 @@ public class TestExpiredMobFileCleaner {
     int timeToLive = expireDays * secondsOfDay();
     hcd.setTimeToLive(timeToLive);
 
-    admin.modifyColumn(tableName, hcd);
+    admin.modifyColumnFamily(tableName, hcd);
   }
 
   private void putKVAndFlush(BufferedMutator table, byte[] row, byte[] value, long ts)

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
index e556a58..f3cfc39 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestFIFOCompactionPolicy.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
@@ -41,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -151,7 +149,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-TTL";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-TTL");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -180,7 +178,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-MinVersion";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-MinVersion");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
@@ -211,7 +209,7 @@ public class TestFIFOCompactionPolicy {
     TEST_UTIL.startMiniCluster(1);
 
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    String tableName = this.tableName.getNameAsString()+"-MinVersion";
+    TableName tableName = TableName.valueOf(getClass().getSimpleName() + "-BlockingStoreFiles");
     if (admin.tableExists(tableName)) {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 9a32b0e..c61b598 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -210,7 +210,7 @@ module Hbase
     def enable(table_name)
       tableExists(table_name)
       return if enabled?(table_name)
-      @admin.enableTable(table_name)
+      @admin.enableTable(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -225,7 +225,7 @@ module Hbase
     def disable(table_name)
       tableExists(table_name)
       return if disabled?(table_name)
-      @admin.disableTable(table_name)
+      @admin.disableTable(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -244,14 +244,15 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Is table disabled?
     def disabled?(table_name)
-      @admin.isTableDisabled(table_name)
+      @admin.isTableDisabled(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
     # Drops a table
     def drop(table_name)
       tableExists(table_name)
-      raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?(table_name)
+      raise ArgumentError, "Table #{table_name} is enabled. Disable it first." if enabled?(
+          table_name)
 
       @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
     end
@@ -447,15 +448,17 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table (deletes all records by recreating the table)
-    def truncate(table_name, conf = @conf)
-      table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
-      raise ArgumentError, "Table #{table_name} is not enabled. Enable it first." unless enabled?(table_name)
+    def truncate(table_name_str, conf = @conf)
+      table_name = TableName.valueOf(table_name_str)
+      table_description = @admin.getTableDescriptor(table_name)
+      raise ArgumentError, "Table #{table_name_str} is not enabled. Enable it first." unless
+          enabled?(table_name_str)
       yield 'Disabling table...' if block_given?
       @admin.disableTable(table_name)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false)
+        @admin.truncateTable(table_name, false)
       rescue => e
         # Handle the compatibility case, where the truncate method doesn't exists on the Master
         raise e unless e.respond_to?(:cause) && e.cause != nil
@@ -463,7 +466,7 @@ module Hbase
         if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
           # Handle the compatibility case, where the truncate method doesn't exists on the Master
           yield 'Dropping table...' if block_given?
-          @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
+          @admin.deleteTable(table_name)
 
           yield 'Creating table...' if block_given?
           @admin.createTable(table_description)
@@ -475,9 +478,10 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Truncates table while maintaing region boundaries (deletes all records by recreating the table)
-    def truncate_preserve(table_name, conf = @conf)
-      h_table = @connection.getTable(TableName.valueOf(table_name))
-      locator = @connection.getRegionLocator(TableName.valueOf(table_name))
+    def truncate_preserve(table_name_str, conf = @conf)
+      table_name = TableName.valueOf(table_name_str)
+      h_table = @connection.getTable(table_name)
+      locator = @connection.getRegionLocator(table_name)
       begin
         splits = locator.getAllRegionLocations().
             map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}.
@@ -486,13 +490,13 @@ module Hbase
         locator.close()
       end
 
-      table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
+      table_description = @admin.getTableDescriptor(table_name)
       yield 'Disabling table...' if block_given?
-      disable(table_name)
+      disable(table_name_str)
 
       begin
         yield 'Truncating table...' if block_given?
-        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true)
+        @admin.truncateTable(table_name, true)
       rescue => e
         # Handle the compatibility case, where the truncate method doesn't exists on the Master
         raise e unless e.respond_to?(:cause) && e.cause != nil
@@ -500,7 +504,7 @@ module Hbase
         if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
           # Handle the compatibility case, where the truncate method doesn't exists on the Master
           yield 'Dropping table...' if block_given?
-          @admin.deleteTable(org.apache.hadoop.hbase.TableName.valueOf(table_name))
+          @admin.deleteTable(table_name)
 
           yield 'Creating table with region boundaries...' if block_given?
           @admin.createTable(table_description, splits)
@@ -534,18 +538,21 @@ module Hbase
 
     #----------------------------------------------------------------------------------------------
     # Change table structure or table options
-    def alter(table_name, wait = true, *args)
+    def alter(table_name_str, wait = true, *args)
       # Table name should be a string
-      raise(ArgumentError, "Table name must be of type String") unless table_name.kind_of?(String)
+      raise(ArgumentError, "Table name must be of type String") unless
+          table_name_str.kind_of?(String)
 
       # Table should exist
-      raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
+      raise(ArgumentError, "Can't find a table: #{table_name_str}") unless exists?(table_name_str)
 
       # There should be at least one argument
       raise(ArgumentError, "There should be at least one argument but the table name") if args.empty?
 
+      table_name = TableName.valueOf(table_name_str)
+
       # Get table descriptor
-      htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
+      htd = @admin.getTableDescriptor(table_name)
 
       # Process all args
       args.each do |arg|
@@ -573,11 +580,11 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
 
           # We bypass descriptor when adding column families; refresh it to apply other args correctly.
-          htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
+          htd = @admin.getTableDescriptor(table_name)
           next
         end
 
@@ -587,7 +594,7 @@ module Hbase
           # Delete column family
           if method == "delete"
             raise(ArgumentError, "NAME parameter missing for delete method") unless name
-            @admin.deleteColumn(table_name, name)
+            @admin.deleteColumn(table_name, name.to_java_bytes)
           # Unset table attributes
           elsif method == "table_att_unset"
             raise(ArgumentError, "NAME parameter missing for table_att_unset method") unless name
@@ -604,7 +611,7 @@ module Hbase
               end
               htd.remove(name)
             end
-            @admin.modifyTable(table_name.to_java_bytes, htd)
+            @admin.modifyTable(table_name, htd)
           # Unknown method
           else
             raise ArgumentError, "Unknown method: #{method}"
@@ -616,12 +623,12 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
 
           if method == "delete"
             # We bypass descriptor when deleting column families; refresh it to apply other args correctly.
-            htd = @admin.getTableDescriptor(TableName.valueOf(table_name))
+            htd = @admin.getTableDescriptor(table_name)
           end
           next
         end
@@ -667,7 +674,7 @@ module Hbase
             arg.delete(key)
           end
 
-          @admin.modifyTable(table_name.to_java_bytes, htd)
+          @admin.modifyTable(table_name, htd)
 
           arg.each_key do |unknown_key|
             puts("Unknown argument ignored: %s" % [unknown_key])
@@ -675,7 +682,7 @@ module Hbase
 
           if wait == true
             puts "Updating all regions with the new schema..."
-            alter_status(table_name)
+            alter_status(table_name_str)
           end
           next
         end
@@ -792,13 +799,13 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
     # Is table enabled
     def enabled?(table_name)
-      @admin.isTableEnabled(table_name)
+      @admin.isTableEnabled(TableName.valueOf(table_name))
     end
 
     #----------------------------------------------------------------------------------------------
@@ -910,14 +917,23 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Take a snapshot of specified table
     def snapshot(table, snapshot_name, *args)
+      # Table name should be a string
+      raise(ArgumentError, "Table name must be of type String") unless table.kind_of?(String)
+
+      # Snapshot name should be a string
+      raise(ArgumentError, "Snapshot name must be of type String") unless
+          snapshot_name.kind_of?(String)
+
+      table_name = TableName.valueOf(table)
       if args.empty?
-         @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
+         @admin.snapshot(snapshot_name, table_name)
       else
          args.each do |arg|
             if arg[SKIP_FLUSH] == true
-              @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes, SnapshotDescription::Type::SKIPFLUSH)
+              @admin.snapshot(snapshot_name, table_name,
+                              SnapshotDescription::Type::SKIPFLUSH)
             else
-               @admin.snapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
+               @admin.snapshot(snapshot_name, table_name)
             end
          end
       end
@@ -926,19 +942,19 @@ module Hbase
     #----------------------------------------------------------------------------------------------
     # Restore specified snapshot
     def restore_snapshot(snapshot_name)
-      @admin.restoreSnapshot(snapshot_name.to_java_bytes)
+      @admin.restoreSnapshot(snapshot_name)
     end
 
     #----------------------------------------------------------------------------------------------
     # Create a new table by cloning the snapshot content
     def clone_snapshot(snapshot_name, table)
-      @admin.cloneSnapshot(snapshot_name.to_java_bytes, table.to_java_bytes)
+      @admin.cloneSnapshot(snapshot_name, TableName.valueOf(table))
     end
 
     #----------------------------------------------------------------------------------------------
     # Delete specified snapshot
     def delete_snapshot(snapshot_name)
-      @admin.deleteSnapshot(snapshot_name.to_java_bytes)
+      @admin.deleteSnapshot(snapshot_name)
     end
 
     #----------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-shell/src/main/ruby/hbase/security.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/security.rb b/hbase-shell/src/main/ruby/hbase/security.rb
index c7b94e8..20f3298 100644
--- a/hbase-shell/src/main/ruby/hbase/security.rb
+++ b/hbase-shell/src/main/ruby/hbase/security.rb
@@ -64,7 +64,7 @@ module Hbase
             # Table should exist
             raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-            tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
+            tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
             htd = @admin.getTableDescriptor(tableName)
 
             if (family != nil)
@@ -106,7 +106,7 @@ module Hbase
              # Table should exist
              raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
 
-             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
+             tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name)
              htd = @admin.getTableDescriptor(tableName)
 
              if (family != nil)
@@ -165,7 +165,7 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
 
     def isNamespace?(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
index 0d50d8a..98bfb2c 100644
--- a/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
+++ b/hbase-shell/src/main/ruby/hbase/visibility_labels.rb
@@ -153,7 +153,7 @@ module Hbase
 
     # Does table exist?
     def exists?(table_name)
-      @admin.tableExists(table_name)
+      @admin.tableExists(TableName.valueOf(table_name))
     end
   end
 end

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-shell/src/test/ruby/hbase/admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/admin_test.rb b/hbase-shell/src/test/ruby/hbase/admin_test.rb
index 0a1a92e..0270037 100644
--- a/hbase-shell/src/test/ruby/hbase/admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/admin_test.rb
@@ -405,21 +405,21 @@ module Hbase
     end
 
     #-------------------------------------------------------------------------------
-    define_test "Snapshot should fail with non-string snapshot name" do
-      assert_raise(NoMethodError) do
+    define_test "Snapshot should fail with non-string table name" do
+      assert_raise(ArgumentError) do
         admin.snapshot(123, 'xxx')
       end
     end
 
-    define_test "Snapshot should fail with non-string table name" do
-      assert_raise(NoMethodError) do
-        admin.snapshot(@create_test_snapshot, 123)
+    define_test "Snapshot should fail with non-string snapshot name" do
+      assert_raise(ArgumentError) do
+        admin.snapshot(@test_name, 123)
       end
     end
 
-    define_test "Snapshot should fail without table name" do
+    define_test "Snapshot should fail without snapshot name" do
       assert_raise(ArgumentError) do
-        admin.snapshot("hbase_create_test_snapshot")
+        admin.snapshot(@test_name)
       end
     end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index aa2ac0f..a4587ec 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -421,7 +421,7 @@ onwards.
 +
 [source,java]
 ----
-String tableName = "users";
+TableName tableName = TableName.valueOf("users");
 String path = "hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar";
 Configuration conf = HBaseConfiguration.create();
 HBaseAdmin admin = new HBaseAdmin(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/src/main/asciidoc/_chapters/external_apis.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/external_apis.adoc b/src/main/asciidoc/_chapters/external_apis.adoc
index 4aa5815..43a428a 100644
--- a/src/main/asciidoc/_chapters/external_apis.adoc
+++ b/src/main/asciidoc/_chapters/external_apis.adoc
@@ -741,7 +741,7 @@ the data, and deletes the table.
 [source,jython]
 ----
 import java.lang
-from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants
+from org.apache.hadoop.hbase import HBaseConfiguration, HTableDescriptor, HColumnDescriptor, HConstants, TableName
 from org.apache.hadoop.hbase.client import HBaseAdmin, HTable, Get
 from org.apache.hadoop.hbase.io import Cell, RowResult
 
@@ -753,7 +753,7 @@ conf = HBaseConfiguration()
 # Create a table named 'test' that has two column families,
 # one named 'content, and the other 'anchor'.  The colons
 # are required for column family names.
-tablename = "test"
+tablename = TableName.valueOf("test")
 
 desc = HTableDescriptor(tablename)
 desc.addFamily(HColumnDescriptor("content:"))

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index db255aa..e8d44eb 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -2214,7 +2214,7 @@ or in code it would be as follows:
 
 [source,java]
 ----
-void rename(Admin admin, String oldTableName, String newTableName) {
+void rename(Admin admin, String oldTableName, TableName newTableName) {
   String snapshotName = randomName();
   admin.disableTable(oldTableName);
   admin.snapshot(snapshotName, oldTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/src/main/asciidoc/_chapters/schema_design.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc
index 926df71..e2dcbad 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -53,7 +53,7 @@ Tables must be disabled when making ColumnFamily modifications, for example:
 
 Configuration config = HBaseConfiguration.create();
 Admin admin = new Admin(conf);
-String table = "myTable";
+TableName table = TableName.valueOf("myTable");
 
 admin.disableTable(table);
 


[07/26] hbase git commit: HBASE-13153 Bulk Loaded HFile Replication (Ashish Singhi)

Posted by sy...@apache.org.
HBASE-13153 Bulk Loaded HFile Replication (Ashish Singhi)


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

Branch: refs/heads/hbase-12439
Commit: 26ac60b03f80c9215103a02db783341e67037753
Parents: 9647fee
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Dec 10 13:07:46 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Dec 10 13:07:46 2015 +0530

----------------------------------------------------------------------
 .../hbase/replication/ReplicationPeers.java     |   2 +-
 .../replication/ReplicationPeersZKImpl.java     |  26 +-
 .../hbase/replication/ReplicationQueues.java    |  25 +-
 .../replication/ReplicationQueuesClient.java    |  25 +-
 .../ReplicationQueuesClientZKImpl.java          |  37 ++
 .../replication/ReplicationQueuesZKImpl.java    |  70 +++
 .../replication/ReplicationStateZKBase.java     |  14 +-
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |  24 +-
 .../org/apache/hadoop/hbase/HConstants.java     |  16 +-
 .../MetricsReplicationSinkSource.java           |   2 +
 .../MetricsReplicationSourceSource.java         |   6 +
 .../MetricsReplicationGlobalSourceSource.java   |  21 +
 .../MetricsReplicationSinkSourceImpl.java       |   7 +
 .../MetricsReplicationSourceSourceImpl.java     |  28 +
 .../hbase/protobuf/generated/AdminProtos.java   | 602 +++++++++++++++++--
 hbase-protocol/src/main/protobuf/Admin.proto    |   3 +
 .../hbase/mapreduce/LoadIncrementalHFiles.java  | 152 +++--
 .../hbase/protobuf/ReplicationProtbufUtil.java  |  46 +-
 .../hbase/regionserver/RSRpcServices.java       |   4 +-
 .../regionserver/ReplicationSinkService.java    |   8 +-
 .../regionserver/wal/WALActionsListener.java    |  19 +-
 .../hbase/replication/ScopeWALEntryFilter.java  |  72 ++-
 .../replication/TableCfWALEntryFilter.java      |  76 ++-
 .../master/ReplicationHFileCleaner.java         | 193 ++++++
 .../DefaultSourceFSConfigurationProvider.java   |  78 +++
 .../HBaseInterClusterReplicationEndpoint.java   |  32 +-
 .../regionserver/HFileReplicator.java           | 393 ++++++++++++
 .../replication/regionserver/MetricsSink.java   |  13 +-
 .../replication/regionserver/MetricsSource.java |  31 +
 .../RegionReplicaReplicationEndpoint.java       |   4 +-
 .../replication/regionserver/Replication.java   | 133 +++-
 .../regionserver/ReplicationSink.java           | 200 +++++-
 .../regionserver/ReplicationSource.java         |  92 ++-
 .../ReplicationSourceInterface.java             |  13 +
 .../regionserver/ReplicationSourceManager.java  |  21 +
 .../SourceFSConfigurationProvider.java          |  40 ++
 .../security/access/SecureBulkLoadEndpoint.java |  18 +-
 .../cleaner/TestReplicationHFileCleaner.java    | 264 ++++++++
 .../replication/ReplicationSourceDummy.java     |   8 +
 .../replication/TestMasterReplication.java      | 313 +++++++++-
 .../replication/TestReplicationSmallTests.java  |   3 +-
 .../replication/TestReplicationStateBasic.java  |  57 ++
 .../replication/TestReplicationStateZKImpl.java |   1 +
 .../replication/TestReplicationSyncUpTool.java  |  10 +-
 ...ReplicationSyncUpToolWithBulkLoadedData.java | 235 ++++++++
 .../regionserver/TestReplicationSink.java       | 179 +++++-
 .../TestReplicationSourceManager.java           |  70 ++-
 .../TestSourceFSConfigurationProvider.java      |  25 +
 48 files changed, 3444 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 8e80e06..8bf21d5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -50,7 +50,7 @@ public interface ReplicationPeers {
    * @param peerId a short that identifies the cluster
    * @param peerConfig configuration for the replication slave cluster
    * @param tableCFs the table and column-family list which will be replicated for this peer or null
-   * for all table and column families
+   *          for all table and column families
    */
   void addPeer(String peerId, ReplicationPeerConfig peerConfig, String tableCFs)
       throws ReplicationException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 63f9ac3..fd10b66 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
 
 import com.google.protobuf.ByteString;
 
@@ -120,8 +121,21 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
 
       checkQueuesDeleted(id);
-      
+
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
+
+      // If only bulk load hfile replication is enabled then add peerId node to hfile-refs node
+      if (replicationForBulkLoadEnabled) {
+        try {
+          String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
+          LOG.info("Adding peer " + peerId + " to hfile reference queue.");
+          ZKUtil.createWithParents(this.zookeeper, peerId);
+        } catch (KeeperException e) {
+          throw new ReplicationException("Failed to add peer with id=" + id
+              + ", node under hfile references node.", e);
+        }
+      }
+
       List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
       ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
         toByteArray(peerConfig));
@@ -151,6 +165,16 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
             + " because that id does not exist.");
       }
       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
+      // Delete peerId node from hfile-refs node irrespective of whether bulk loaded hfile
+      // replication is enabled or not
+
+      String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
+      try {
+        LOG.info("Removing peer " + peerId + " from hfile reference queue.");
+        ZKUtil.deleteNodeRecursively(this.zookeeper, peerId);
+      } catch (NoNodeException e) {
+        LOG.info("Did not find node " + peerId + " to delete.", e);
+      }
     } catch (KeeperException e) {
       throw new ReplicationException("Could not remove peer with id=" + id, e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 3dbbc33..0d47a88 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * This provides an interface for maintaining a region server's replication queues. These queues
- * keep track of the WALs that still need to be replicated to remote clusters.
+ * keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled)
+ * that still need to be replicated to remote clusters.
  */
 @InterfaceAudience.Private
 public interface ReplicationQueues {
@@ -113,4 +114,26 @@ public interface ReplicationQueues {
    * @return if this is this rs's znode
    */
   boolean isThisOurZnode(String znode);
+
+  /**
+   * Add a peer to hfile reference queue if peer does not exist.
+   * @param peerId peer cluster id to be added
+   * @throws ReplicationException if fails to add a peer id to hfile reference queue
+   */
+  void addPeerToHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Add new hfile references to the queue.
+   * @param peerId peer cluster id to which the hfiles need to be replicated
+   * @param files list of hfile references to be added
+   * @throws ReplicationException if fails to add a hfile reference
+   */
+  void addHFileRefs(String peerId, List<String> files) throws ReplicationException;
+
+  /**
+   * Remove hfile references from the queue.
+   * @param peerId peer cluster id from which this hfile references needs to be removed
+   * @param files list of hfile references to be removed
+   */
+  void removeHFileRefs(String peerId, List<String> files);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
index 5b3e541..7fa3bbb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
@@ -25,7 +25,8 @@ import org.apache.zookeeper.KeeperException;
 
 /**
  * This provides an interface for clients of replication to view replication queues. These queues
- * keep track of the WALs that still need to be replicated to remote clusters.
+ * keep track of the sources(WALs/HFile references) that still need to be replicated to remote
+ * clusters.
  */
 @InterfaceAudience.Private
 public interface ReplicationQueuesClient {
@@ -65,4 +66,26 @@ public interface ReplicationQueuesClient {
    * @return cversion of replication rs node
    */
   int getQueuesZNodeCversion() throws KeeperException;
+
+  /**
+   * Get the change version number of replication hfile references node. This can be used as
+   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
+   * @return change version number of hfile references node
+   */
+  int getHFileRefsNodeChangeVersion() throws KeeperException;
+
+  /**
+   * Get list of all peers from hfile reference queue.
+   * @return a list of peer ids
+   * @throws KeeperException zookeeper exception
+   */
+  List<String> getAllPeersFromHFileRefsQueue() throws KeeperException;
+
+  /**
+   * Get a list of all hfile references in the given peer.
+   * @param peerId a String that identifies the peer
+   * @return a list of hfile references, null if not found any
+   * @throws KeeperException zookeeper exception
+   */
+  List<String> getReplicableHFiles(String peerId) throws KeeperException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index e1a6a49..cc407e3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -84,4 +84,41 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
       throw e;
     }
   }
+
+  @Override
+  public int getHFileRefsNodeChangeVersion() throws KeeperException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      this.abortable.abort("Failed to get stat of replication hfile references node.", e);
+      throw e;
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
+    List<String> result = null;
+    try {
+      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
+    } catch (KeeperException e) {
+      this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
+      throw e;
+    }
+    return result;
+  }
+
+  @Override
+  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
+    String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
+    List<String> result = null;
+    try {
+      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
+    } catch (KeeperException e) {
+      this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
+      throw e;
+    }
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 97763e2..43dd412 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -84,6 +84,15 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
     } catch (KeeperException e) {
       throw new ReplicationException("Could not initialize replication queues.", e);
     }
+    // If only bulk load hfile replication is enabled then create the hfile-refs znode
+    if (replicationForBulkLoadEnabled) {
+      try {
+        ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
+      } catch (KeeperException e) {
+        throw new ReplicationException("Could not initialize hfile references replication queue.",
+            e);
+      }
+    }
   }
 
   @Override
@@ -431,4 +440,65 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
         ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
     return ProtobufUtil.prependPBMagic(bytes);
   }
+
+  @Override
+  public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
+    String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Adding hfile references " + files + " in queue " + peerZnode);
+    }
+    List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
+    int size = files.size();
+    for (int i = 0; i < size; i++) {
+      listOfOps.add(ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)),
+        HConstants.EMPTY_BYTE_ARRAY));
+    }
+    if (debugEnabled) {
+      LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
+          + " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
+    }
+  }
+
+  @Override
+  public void removeHFileRefs(String peerId, List<String> files) {
+    String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
+    }
+    List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
+    int size = files.size();
+    for (int i = 0; i < size; i++) {
+      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));
+    }
+    if (debugEnabled) {
+      LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
+          + " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
+    }
+  }
+
+  @Override
+  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+    String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
+    try {
+      if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
+        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
+        ZKUtil.createWithParents(this.zookeeper, peerZnode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
+          e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 4fbac0f..762167f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@@ -48,32 +49,43 @@ public abstract class ReplicationStateZKBase {
   protected final String peersZNode;
   /** The name of the znode that contains all replication queues */
   protected final String queuesZNode;
+  /** The name of the znode that contains queues of hfile references to be replicated */
+  protected final String hfileRefsZNode;
   /** The cluster key of the local cluster */
   protected final String ourClusterKey;
   protected final ZooKeeperWatcher zookeeper;
   protected final Configuration conf;
   protected final Abortable abortable;
+  protected final boolean replicationForBulkLoadEnabled;
 
   // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
       toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =
       toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
+      "zookeeper.znode.replication.hfile.refs";
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
 
   public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
       Abortable abortable) {
     this.zookeeper = zookeeper;
     this.conf = conf;
     this.abortable = abortable;
+    this.replicationForBulkLoadEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
 
     String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
     String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
     String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
+    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
+      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
     this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
     this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
     this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
     this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
     this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
+    this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
   }
 
   public List<String> getListOfReplicators() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index c268268..9e01d09 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -884,7 +885,7 @@ public class ZKUtil {
               JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
           && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
           && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
-              
+
         return false;
       }
     } catch(Exception e) {
@@ -1797,6 +1798,27 @@ public class ZKUtil {
       } else if (child.equals(zkw.getConfiguration().
           get("zookeeper.znode.replication.rs", "rs"))) {
         appendRSZnodes(zkw, znode, sb);
+      } else if (child.equals(zkw.getConfiguration().get(
+        ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
+        ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT))) {
+        appendHFileRefsZnodes(zkw, znode, sb);
+      }
+    }
+  }
+
+  private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode,
+      StringBuilder sb) throws KeeperException {
+    sb.append("\n").append(hfileRefsZnode).append(": ");
+    for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
+      String znodeToProcess = ZKUtil.joinZNode(hfileRefsZnode, peerIdZnode);
+      sb.append("\n").append(znodeToProcess).append(": ");
+      List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
+      int size = peerHFileRefsZnodes.size();
+      for (int i = 0; i < size; i++) {
+        sb.append(peerHFileRefsZnodes.get(i));
+        if (i != size - 1) {
+          sb.append(", ");
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index ac57514..6fafad3 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -850,6 +850,18 @@ public final class HConstants {
       REPLICATION_SINK_SERVICE_CLASSNAME = "hbase.replication.sink.service";
   public static final String REPLICATION_SERVICE_CLASSNAME_DEFAULT =
     "org.apache.hadoop.hbase.replication.regionserver.Replication";
+  public static final String REPLICATION_BULKLOAD_ENABLE_KEY = "hbase.replication.bulkload.enabled";
+  public static final boolean REPLICATION_BULKLOAD_ENABLE_DEFAULT = false;
+  /** Replication cluster id of source cluster which uniquely identifies itself with peer cluster */
+  public static final String REPLICATION_CLUSTER_ID = "hbase.replication.cluster.id";
+  /**
+   * Directory where the source cluster file system client configuration are placed which is used by
+   * sink cluster to copy HFiles from source cluster file system
+   */
+  public static final String REPLICATION_CONF_DIR = "hbase.replication.conf.dir";
+
+  /** Maximum time to retry for a failed bulk load request */
+  public static final String BULKLOAD_MAX_RETRIES_NUMBER = "hbase.bulkload.retries.number";
 
   /** HBCK special code name used as server name when manipulating ZK nodes */
   public static final String HBCK_CODE_NAME = "HBCKServerName";
@@ -1241,7 +1253,7 @@ public final class HConstants {
 
   public static final String HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY =
       "hbase.canary.write.table.check.period";
-  
+
   /**
    * Configuration keys for programmatic JAAS configuration for secured ZK interaction
    */
@@ -1250,7 +1262,7 @@ public final class HConstants {
       "hbase.zookeeper.client.kerberos.principal";
   public static final String ZK_SERVER_KEYTAB_FILE = "hbase.zookeeper.server.keytab.file";
   public static final String ZK_SERVER_KERBEROS_PRINCIPAL =
-      "hbase.zookeeper.server.kerberos.principal";  
+      "hbase.zookeeper.server.kerberos.principal";
 
   private HConstants() {
     // Can't be instantiated with this ctor.

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
index 698a59a..9fb8415 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSource.java
@@ -22,9 +22,11 @@ public interface MetricsReplicationSinkSource {
   public static final String SINK_AGE_OF_LAST_APPLIED_OP = "sink.ageOfLastAppliedOp";
   public static final String SINK_APPLIED_BATCHES = "sink.appliedBatches";
   public static final String SINK_APPLIED_OPS = "sink.appliedOps";
+  public static final String SINK_APPLIED_HFILES = "sink.appliedHFiles";
 
   void setLastAppliedOpAge(long age);
   void incrAppliedBatches(long batches);
   void incrAppliedOps(long batchsize);
   long getLastAppliedOpAge();
+  void incrAppliedHFiles(long hfileSize);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
index fecf191..188c3a3 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSource.java
@@ -32,6 +32,9 @@ public interface MetricsReplicationSourceSource {
 
   public static final String SOURCE_LOG_EDITS_FILTERED = "source.logEditsFiltered";
 
+  public static final String SOURCE_SHIPPED_HFILES = "source.shippedHFiles";
+  public static final String SOURCE_SIZE_OF_HFILE_REFS_QUEUE = "source.sizeOfHFileRefsQueue";
+
   void setLastShippedAge(long age);
   void setSizeOfLogQueue(int size);
   void incrSizeOfLogQueue(int size);
@@ -44,4 +47,7 @@ public interface MetricsReplicationSourceSource {
   void incrLogReadInEdits(long size);
   void clear();
   long getLastShippedAge();
+  void incrHFilesShipped(long hfiles);
+  void incrSizeOfHFileRefsQueue(long size);
+  void decrSizeOfHFileRefsQueue(long size);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
index 6dace10..392cd39 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationGlobalSourceSource.java
@@ -32,6 +32,8 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
   private final MutableCounterLong shippedOpsCounter;
   private final MutableCounterLong shippedKBsCounter;
   private final MutableCounterLong logReadInBytesCounter;
+  private final MutableCounterLong shippedHFilesCounter;
+  private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
 
   public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
     this.rms = rms;
@@ -51,6 +53,11 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
     logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
 
     logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
+
+    shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_HFILES, 0L);
+
+    sizeOfHFileRefsQueueGauge =
+        rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
   }
 
   @Override public void setLastShippedAge(long age) {
@@ -100,4 +107,18 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
   public long getLastShippedAge() {
     return ageOfLastShippedOpGauge.value();
   }
+
+  @Override public void incrHFilesShipped(long hfiles) {
+    shippedHFilesCounter.incr(hfiles);
+  }
+
+  @Override
+  public void incrSizeOfHFileRefsQueue(long size) {
+    sizeOfHFileRefsQueueGauge.incr(size);
+  }
+
+  @Override
+  public void decrSizeOfHFileRefsQueue(long size) {
+    sizeOfHFileRefsQueueGauge.decr(size);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
index 14212ba..8f4a337 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSinkSourceImpl.java
@@ -26,11 +26,13 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
   private final MutableGaugeLong ageGauge;
   private final MutableCounterLong batchesCounter;
   private final MutableCounterLong opsCounter;
+  private final MutableCounterLong hfilesCounter;
 
   public MetricsReplicationSinkSourceImpl(MetricsReplicationSourceImpl rms) {
     ageGauge = rms.getMetricsRegistry().getLongGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
     batchesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_BATCHES, 0L);
     opsCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_OPS, 0L);
+    hfilesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_HFILES, 0L);
   }
 
   @Override public void setLastAppliedOpAge(long age) {
@@ -49,4 +51,9 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
   public long getLastAppliedOpAge() {
     return ageGauge.value();
   }
+
+  @Override
+  public void incrAppliedHFiles(long hfiles) {
+    hfilesCounter.incr(hfiles);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
index 1422e7e..217cc3e 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/replication/regionserver/MetricsReplicationSourceSourceImpl.java
@@ -32,6 +32,8 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
   private final String shippedOpsKey;
   private final String shippedKBsKey;
   private final String logReadInBytesKey;
+  private final String shippedHFilesKey;
+  private final String sizeOfHFileRefsQueueKey;
 
   private final MutableGaugeLong ageOfLastShippedOpGauge;
   private final MutableGaugeLong sizeOfLogQueueGauge;
@@ -41,6 +43,8 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
   private final MutableCounterLong shippedOpsCounter;
   private final MutableCounterLong shippedKBsCounter;
   private final MutableCounterLong logReadInBytesCounter;
+  private final MutableCounterLong shippedHFilesCounter;
+  private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
 
   public MetricsReplicationSourceSourceImpl(MetricsReplicationSourceImpl rms, String id) {
     this.rms = rms;
@@ -69,6 +73,12 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
 
     logEditsFilteredKey = "source." + id + ".logEditsFiltered";
     logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(logEditsFilteredKey, 0L);
+
+    shippedHFilesKey = "source." + this.id + ".shippedHFiles";
+    shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(shippedHFilesKey, 0L);
+
+    sizeOfHFileRefsQueueKey = "source." + id + ".sizeOfHFileRefsQueue";
+    sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfHFileRefsQueueKey, 0L);
   }
 
   @Override public void setLastShippedAge(long age) {
@@ -124,10 +134,28 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
     rms.removeMetric(logReadInEditsKey);
 
     rms.removeMetric(logEditsFilteredKey);
+
+    rms.removeMetric(shippedHFilesKey);
+    rms.removeMetric(sizeOfHFileRefsQueueKey);
   }
 
   @Override
   public long getLastShippedAge() {
     return ageOfLastShippedOpGauge.value();
   }
+
+  @Override
+  public void incrHFilesShipped(long hfiles) {
+    shippedHFilesCounter.incr(hfiles);
+  }
+
+  @Override
+  public void incrSizeOfHFileRefsQueue(long size) {
+    sizeOfHFileRefsQueueGauge.incr(size);
+  }
+
+  @Override
+  public void decrSizeOfHFileRefsQueue(long size) {
+    sizeOfHFileRefsQueueGauge.decr(size);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
index b4c378b..1c59ea6 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java
@@ -16896,6 +16896,51 @@ public final class AdminProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntryOrBuilder getEntryOrBuilder(
         int index);
+
+    // optional string replicationClusterId = 2;
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    boolean hasReplicationClusterId();
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    java.lang.String getReplicationClusterId();
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getReplicationClusterIdBytes();
+
+    // optional string sourceBaseNamespaceDirPath = 3;
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    boolean hasSourceBaseNamespaceDirPath();
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    java.lang.String getSourceBaseNamespaceDirPath();
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    com.google.protobuf.ByteString
+        getSourceBaseNamespaceDirPathBytes();
+
+    // optional string sourceHFileArchiveDirPath = 4;
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    boolean hasSourceHFileArchiveDirPath();
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    java.lang.String getSourceHFileArchiveDirPath();
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getSourceHFileArchiveDirPathBytes();
   }
   /**
    * Protobuf type {@code hbase.pb.ReplicateWALEntryRequest}
@@ -16963,6 +17008,21 @@ public final class AdminProtos {
               entry_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry.PARSER, extensionRegistry));
               break;
             }
+            case 18: {
+              bitField0_ |= 0x00000001;
+              replicationClusterId_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000002;
+              sourceBaseNamespaceDirPath_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000004;
+              sourceHFileArchiveDirPath_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -17005,6 +17065,7 @@ public final class AdminProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated .hbase.pb.WALEntry entry = 1;
     public static final int ENTRY_FIELD_NUMBER = 1;
     private java.util.List<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry> entry_;
@@ -17041,8 +17102,140 @@ public final class AdminProtos {
       return entry_.get(index);
     }
 
+    // optional string replicationClusterId = 2;
+    public static final int REPLICATIONCLUSTERID_FIELD_NUMBER = 2;
+    private java.lang.Object replicationClusterId_;
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    public boolean hasReplicationClusterId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    public java.lang.String getReplicationClusterId() {
+      java.lang.Object ref = replicationClusterId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          replicationClusterId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string replicationClusterId = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getReplicationClusterIdBytes() {
+      java.lang.Object ref = replicationClusterId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        replicationClusterId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string sourceBaseNamespaceDirPath = 3;
+    public static final int SOURCEBASENAMESPACEDIRPATH_FIELD_NUMBER = 3;
+    private java.lang.Object sourceBaseNamespaceDirPath_;
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    public boolean hasSourceBaseNamespaceDirPath() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    public java.lang.String getSourceBaseNamespaceDirPath() {
+      java.lang.Object ref = sourceBaseNamespaceDirPath_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          sourceBaseNamespaceDirPath_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+     */
+    public com.google.protobuf.ByteString
+        getSourceBaseNamespaceDirPathBytes() {
+      java.lang.Object ref = sourceBaseNamespaceDirPath_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        sourceBaseNamespaceDirPath_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string sourceHFileArchiveDirPath = 4;
+    public static final int SOURCEHFILEARCHIVEDIRPATH_FIELD_NUMBER = 4;
+    private java.lang.Object sourceHFileArchiveDirPath_;
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    public boolean hasSourceHFileArchiveDirPath() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    public java.lang.String getSourceHFileArchiveDirPath() {
+      java.lang.Object ref = sourceHFileArchiveDirPath_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          sourceHFileArchiveDirPath_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getSourceHFileArchiveDirPathBytes() {
+      java.lang.Object ref = sourceHFileArchiveDirPath_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        sourceHFileArchiveDirPath_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
     private void initFields() {
       entry_ = java.util.Collections.emptyList();
+      replicationClusterId_ = "";
+      sourceBaseNamespaceDirPath_ = "";
+      sourceHFileArchiveDirPath_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -17065,6 +17258,15 @@ public final class AdminProtos {
       for (int i = 0; i < entry_.size(); i++) {
         output.writeMessage(1, entry_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(2, getReplicationClusterIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(3, getSourceBaseNamespaceDirPathBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(4, getSourceHFileArchiveDirPathBytes());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -17078,6 +17280,18 @@ public final class AdminProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, entry_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getReplicationClusterIdBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getSourceBaseNamespaceDirPathBytes());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getSourceHFileArchiveDirPathBytes());
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -17103,6 +17317,21 @@ public final class AdminProtos {
       boolean result = true;
       result = result && getEntryList()
           .equals(other.getEntryList());
+      result = result && (hasReplicationClusterId() == other.hasReplicationClusterId());
+      if (hasReplicationClusterId()) {
+        result = result && getReplicationClusterId()
+            .equals(other.getReplicationClusterId());
+      }
+      result = result && (hasSourceBaseNamespaceDirPath() == other.hasSourceBaseNamespaceDirPath());
+      if (hasSourceBaseNamespaceDirPath()) {
+        result = result && getSourceBaseNamespaceDirPath()
+            .equals(other.getSourceBaseNamespaceDirPath());
+      }
+      result = result && (hasSourceHFileArchiveDirPath() == other.hasSourceHFileArchiveDirPath());
+      if (hasSourceHFileArchiveDirPath()) {
+        result = result && getSourceHFileArchiveDirPath()
+            .equals(other.getSourceHFileArchiveDirPath());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -17120,6 +17349,18 @@ public final class AdminProtos {
         hash = (37 * hash) + ENTRY_FIELD_NUMBER;
         hash = (53 * hash) + getEntryList().hashCode();
       }
+      if (hasReplicationClusterId()) {
+        hash = (37 * hash) + REPLICATIONCLUSTERID_FIELD_NUMBER;
+        hash = (53 * hash) + getReplicationClusterId().hashCode();
+      }
+      if (hasSourceBaseNamespaceDirPath()) {
+        hash = (37 * hash) + SOURCEBASENAMESPACEDIRPATH_FIELD_NUMBER;
+        hash = (53 * hash) + getSourceBaseNamespaceDirPath().hashCode();
+      }
+      if (hasSourceHFileArchiveDirPath()) {
+        hash = (37 * hash) + SOURCEHFILEARCHIVEDIRPATH_FIELD_NUMBER;
+        hash = (53 * hash) + getSourceHFileArchiveDirPath().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -17243,6 +17484,12 @@ public final class AdminProtos {
         } else {
           entryBuilder_.clear();
         }
+        replicationClusterId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        sourceBaseNamespaceDirPath_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        sourceHFileArchiveDirPath_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -17270,6 +17517,7 @@ public final class AdminProtos {
       public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (entryBuilder_ == null) {
           if (((bitField0_ & 0x00000001) == 0x00000001)) {
             entry_ = java.util.Collections.unmodifiableList(entry_);
@@ -17279,6 +17527,19 @@ public final class AdminProtos {
         } else {
           result.entry_ = entryBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.replicationClusterId_ = replicationClusterId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.sourceBaseNamespaceDirPath_ = sourceBaseNamespaceDirPath_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.sourceHFileArchiveDirPath_ = sourceHFileArchiveDirPath_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -17320,6 +17581,21 @@ public final class AdminProtos {
             }
           }
         }
+        if (other.hasReplicationClusterId()) {
+          bitField0_ |= 0x00000002;
+          replicationClusterId_ = other.replicationClusterId_;
+          onChanged();
+        }
+        if (other.hasSourceBaseNamespaceDirPath()) {
+          bitField0_ |= 0x00000004;
+          sourceBaseNamespaceDirPath_ = other.sourceBaseNamespaceDirPath_;
+          onChanged();
+        }
+        if (other.hasSourceHFileArchiveDirPath()) {
+          bitField0_ |= 0x00000008;
+          sourceHFileArchiveDirPath_ = other.sourceHFileArchiveDirPath_;
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -17593,6 +17869,228 @@ public final class AdminProtos {
         return entryBuilder_;
       }
 
+      // optional string replicationClusterId = 2;
+      private java.lang.Object replicationClusterId_ = "";
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public boolean hasReplicationClusterId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public java.lang.String getReplicationClusterId() {
+        java.lang.Object ref = replicationClusterId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          replicationClusterId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getReplicationClusterIdBytes() {
+        java.lang.Object ref = replicationClusterId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          replicationClusterId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public Builder setReplicationClusterId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationClusterId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public Builder clearReplicationClusterId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        replicationClusterId_ = getDefaultInstance().getReplicationClusterId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationClusterId = 2;</code>
+       */
+      public Builder setReplicationClusterIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationClusterId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string sourceBaseNamespaceDirPath = 3;
+      private java.lang.Object sourceBaseNamespaceDirPath_ = "";
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public boolean hasSourceBaseNamespaceDirPath() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public java.lang.String getSourceBaseNamespaceDirPath() {
+        java.lang.Object ref = sourceBaseNamespaceDirPath_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          sourceBaseNamespaceDirPath_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public com.google.protobuf.ByteString
+          getSourceBaseNamespaceDirPathBytes() {
+        java.lang.Object ref = sourceBaseNamespaceDirPath_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          sourceBaseNamespaceDirPath_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public Builder setSourceBaseNamespaceDirPath(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        sourceBaseNamespaceDirPath_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public Builder clearSourceBaseNamespaceDirPath() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        sourceBaseNamespaceDirPath_ = getDefaultInstance().getSourceBaseNamespaceDirPath();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string sourceBaseNamespaceDirPath = 3;</code>
+       */
+      public Builder setSourceBaseNamespaceDirPathBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        sourceBaseNamespaceDirPath_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string sourceHFileArchiveDirPath = 4;
+      private java.lang.Object sourceHFileArchiveDirPath_ = "";
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public boolean hasSourceHFileArchiveDirPath() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public java.lang.String getSourceHFileArchiveDirPath() {
+        java.lang.Object ref = sourceHFileArchiveDirPath_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          sourceHFileArchiveDirPath_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getSourceHFileArchiveDirPathBytes() {
+        java.lang.Object ref = sourceHFileArchiveDirPath_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          sourceHFileArchiveDirPath_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public Builder setSourceHFileArchiveDirPath(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        sourceHFileArchiveDirPath_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public Builder clearSourceHFileArchiveDirPath() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        sourceHFileArchiveDirPath_ = getDefaultInstance().getSourceHFileArchiveDirPath();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string sourceHFileArchiveDirPath = 4;</code>
+       */
+      public Builder setSourceHFileArchiveDirPathBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        sourceHFileArchiveDirPath_ = value;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicateWALEntryRequest)
     }
 
@@ -23539,56 +24037,58 @@ public final class AdminProtos {
       "ster_system_time\030\004 \001(\004\"\026\n\024MergeRegionsRe" +
       "sponse\"a\n\010WALEntry\022\035\n\003key\030\001 \002(\0132\020.hbase." +
       "pb.WALKey\022\027\n\017key_value_bytes\030\002 \003(\014\022\035\n\025as",
-      "sociated_cell_count\030\003 \001(\005\"=\n\030ReplicateWA" +
-      "LEntryRequest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb." +
-      "WALEntry\"\033\n\031ReplicateWALEntryResponse\"\026\n" +
-      "\024RollWALWriterRequest\"0\n\025RollWALWriterRe" +
-      "sponse\022\027\n\017region_to_flush\030\001 \003(\014\"#\n\021StopS" +
-      "erverRequest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServ" +
-      "erResponse\"\026\n\024GetServerInfoRequest\"K\n\nSe" +
-      "rverInfo\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb" +
-      ".ServerName\022\022\n\nwebui_port\030\002 \001(\r\"B\n\025GetSe" +
-      "rverInfoResponse\022)\n\013server_info\030\001 \002(\0132\024.",
-      "hbase.pb.ServerInfo\"\034\n\032UpdateConfigurati" +
-      "onRequest\"\035\n\033UpdateConfigurationResponse" +
-      "2\207\013\n\014AdminService\022P\n\rGetRegionInfo\022\036.hba" +
-      "se.pb.GetRegionInfoRequest\032\037.hbase.pb.Ge" +
-      "tRegionInfoResponse\022M\n\014GetStoreFile\022\035.hb" +
-      "ase.pb.GetStoreFileRequest\032\036.hbase.pb.Ge" +
-      "tStoreFileResponse\022V\n\017GetOnlineRegion\022 ." +
-      "hbase.pb.GetOnlineRegionRequest\032!.hbase." +
-      "pb.GetOnlineRegionResponse\022G\n\nOpenRegion" +
-      "\022\033.hbase.pb.OpenRegionRequest\032\034.hbase.pb",
-      ".OpenRegionResponse\022M\n\014WarmupRegion\022\035.hb" +
-      "ase.pb.WarmupRegionRequest\032\036.hbase.pb.Wa" +
-      "rmupRegionResponse\022J\n\013CloseRegion\022\034.hbas" +
-      "e.pb.CloseRegionRequest\032\035.hbase.pb.Close" +
-      "RegionResponse\022J\n\013FlushRegion\022\034.hbase.pb" +
-      ".FlushRegionRequest\032\035.hbase.pb.FlushRegi" +
-      "onResponse\022J\n\013SplitRegion\022\034.hbase.pb.Spl" +
-      "itRegionRequest\032\035.hbase.pb.SplitRegionRe" +
-      "sponse\022P\n\rCompactRegion\022\036.hbase.pb.Compa" +
-      "ctRegionRequest\032\037.hbase.pb.CompactRegion",
-      "Response\022M\n\014MergeRegions\022\035.hbase.pb.Merg" +
-      "eRegionsRequest\032\036.hbase.pb.MergeRegionsR" +
-      "esponse\022\\\n\021ReplicateWALEntry\022\".hbase.pb." +
-      "ReplicateWALEntryRequest\032#.hbase.pb.Repl" +
-      "icateWALEntryResponse\022Q\n\006Replay\022\".hbase." +
-      "pb.ReplicateWALEntryRequest\032#.hbase.pb.R" +
-      "eplicateWALEntryResponse\022P\n\rRollWALWrite" +
-      "r\022\036.hbase.pb.RollWALWriterRequest\032\037.hbas" +
-      "e.pb.RollWALWriterResponse\022P\n\rGetServerI" +
-      "nfo\022\036.hbase.pb.GetServerInfoRequest\032\037.hb",
-      "ase.pb.GetServerInfoResponse\022G\n\nStopServ" +
-      "er\022\033.hbase.pb.StopServerRequest\032\034.hbase." +
-      "pb.StopServerResponse\022_\n\022UpdateFavoredNo" +
-      "des\022#.hbase.pb.UpdateFavoredNodesRequest" +
-      "\032$.hbase.pb.UpdateFavoredNodesResponse\022b" +
-      "\n\023UpdateConfiguration\022$.hbase.pb.UpdateC" +
-      "onfigurationRequest\032%.hbase.pb.UpdateCon" +
-      "figurationResponseBA\n*org.apache.hadoop." +
-      "hbase.protobuf.generatedB\013AdminProtosH\001\210" +
-      "\001\001\240\001\001"
+      "sociated_cell_count\030\003 \001(\005\"\242\001\n\030ReplicateW" +
+      "ALEntryRequest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb" +
+      ".WALEntry\022\034\n\024replicationClusterId\030\002 \001(\t\022" +
+      "\"\n\032sourceBaseNamespaceDirPath\030\003 \001(\t\022!\n\031s" +
+      "ourceHFileArchiveDirPath\030\004 \001(\t\"\033\n\031Replic" +
+      "ateWALEntryResponse\"\026\n\024RollWALWriterRequ" +
+      "est\"0\n\025RollWALWriterResponse\022\027\n\017region_t" +
+      "o_flush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006re" +
+      "ason\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024GetS" +
+      "erverInfoRequest\"K\n\nServerInfo\022)\n\013server",
+      "_name\030\001 \002(\0132\024.hbase.pb.ServerName\022\022\n\nweb" +
+      "ui_port\030\002 \001(\r\"B\n\025GetServerInfoResponse\022)" +
+      "\n\013server_info\030\001 \002(\0132\024.hbase.pb.ServerInf" +
+      "o\"\034\n\032UpdateConfigurationRequest\"\035\n\033Updat" +
+      "eConfigurationResponse2\207\013\n\014AdminService\022" +
+      "P\n\rGetRegionInfo\022\036.hbase.pb.GetRegionInf" +
+      "oRequest\032\037.hbase.pb.GetRegionInfoRespons" +
+      "e\022M\n\014GetStoreFile\022\035.hbase.pb.GetStoreFil" +
+      "eRequest\032\036.hbase.pb.GetStoreFileResponse" +
+      "\022V\n\017GetOnlineRegion\022 .hbase.pb.GetOnline",
+      "RegionRequest\032!.hbase.pb.GetOnlineRegion" +
+      "Response\022G\n\nOpenRegion\022\033.hbase.pb.OpenRe" +
+      "gionRequest\032\034.hbase.pb.OpenRegionRespons" +
+      "e\022M\n\014WarmupRegion\022\035.hbase.pb.WarmupRegio" +
+      "nRequest\032\036.hbase.pb.WarmupRegionResponse" +
+      "\022J\n\013CloseRegion\022\034.hbase.pb.CloseRegionRe" +
+      "quest\032\035.hbase.pb.CloseRegionResponse\022J\n\013" +
+      "FlushRegion\022\034.hbase.pb.FlushRegionReques" +
+      "t\032\035.hbase.pb.FlushRegionResponse\022J\n\013Spli" +
+      "tRegion\022\034.hbase.pb.SplitRegionRequest\032\035.",
+      "hbase.pb.SplitRegionResponse\022P\n\rCompactR" +
+      "egion\022\036.hbase.pb.CompactRegionRequest\032\037." +
+      "hbase.pb.CompactRegionResponse\022M\n\014MergeR" +
+      "egions\022\035.hbase.pb.MergeRegionsRequest\032\036." +
+      "hbase.pb.MergeRegionsResponse\022\\\n\021Replica" +
+      "teWALEntry\022\".hbase.pb.ReplicateWALEntryR" +
+      "equest\032#.hbase.pb.ReplicateWALEntryRespo" +
+      "nse\022Q\n\006Replay\022\".hbase.pb.ReplicateWALEnt" +
+      "ryRequest\032#.hbase.pb.ReplicateWALEntryRe" +
+      "sponse\022P\n\rRollWALWriter\022\036.hbase.pb.RollW",
+      "ALWriterRequest\032\037.hbase.pb.RollWALWriter" +
+      "Response\022P\n\rGetServerInfo\022\036.hbase.pb.Get" +
+      "ServerInfoRequest\032\037.hbase.pb.GetServerIn" +
+      "foResponse\022G\n\nStopServer\022\033.hbase.pb.Stop" +
+      "ServerRequest\032\034.hbase.pb.StopServerRespo" +
+      "nse\022_\n\022UpdateFavoredNodes\022#.hbase.pb.Upd" +
+      "ateFavoredNodesRequest\032$.hbase.pb.Update" +
+      "FavoredNodesResponse\022b\n\023UpdateConfigurat" +
+      "ion\022$.hbase.pb.UpdateConfigurationReques" +
+      "t\032%.hbase.pb.UpdateConfigurationResponse",
+      "BA\n*org.apache.hadoop.hbase.protobuf.gen" +
+      "eratedB\013AdminProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -23750,7 +24250,7 @@ public final class AdminProtos {
           internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor,
-              new java.lang.String[] { "Entry", });
+              new java.lang.String[] { "Entry", "ReplicationClusterId", "SourceBaseNamespaceDirPath", "SourceHFileArchiveDirPath", });
           internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor =
             getDescriptor().getMessageTypes().get(24);
           internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-protocol/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Admin.proto b/hbase-protocol/src/main/protobuf/Admin.proto
index f7787f5..a1905a4 100644
--- a/hbase-protocol/src/main/protobuf/Admin.proto
+++ b/hbase-protocol/src/main/protobuf/Admin.proto
@@ -211,6 +211,9 @@ message WALEntry {
  */
 message ReplicateWALEntryRequest {
   repeated WALEntry entry = 1;
+  optional string replicationClusterId = 2;
+  optional string sourceBaseNamespaceDirPath = 3;
+  optional string sourceHFileArchiveDirPath = 4;
 }
 
 message ReplicateWALEntryResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 44be2d3..369ae90 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint;
 import org.apache.hadoop.hbase.security.token.FsDelegationToken;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSHDFSUtils;
@@ -125,6 +126,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   private FsDelegationToken fsDelegationToken;
   private String bulkToken;
   private UserProvider userProvider;
+  private int nrThreads;
 
   private LoadIncrementalHFiles() {}
 
@@ -146,6 +148,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     this.fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
     assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
     maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
+    nrThreads = conf.getInt("hbase.loadincremental.threads.max",
+      Runtime.getRuntime().availableProcessors());
     initalized = true;
   }
 
@@ -246,7 +250,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
    * region boundary, and each part is added back into the queue.
    * The import process finishes when the queue is empty.
    */
-  static class LoadQueueItem {
+  public static class LoadQueueItem {
     final byte[] family;
     final Path hfilePath;
 
@@ -313,7 +317,6 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
    * @param table the table to load into
    * @throws TableNotFoundException if table does not yet exist
    */
-  @SuppressWarnings("deprecation")
   public void doBulkLoad(Path hfofDir, final Admin admin, Table table,
       RegionLocator regionLocator) throws TableNotFoundException, IOException  {
 
@@ -321,16 +324,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       throw new TableNotFoundException("Table " + table.getName() + "is not currently available.");
     }
 
-    // initialize thread pools
-    int nrThreads = getConf().getInt("hbase.loadincremental.threads.max",
-      Runtime.getRuntime().availableProcessors());
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    builder.setNameFormat("LoadIncrementalHFiles-%1$d");
-    ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
-        60, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<Runnable>(),
-        builder.build());
-    ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
+    ExecutorService pool = createExecutorService();
 
     // LQI queue does not need to be threadsafe -- all operations on this queue
     // happen in this thread
@@ -347,30 +341,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 	    "option, consider removing the files and bulkload again without this option. " +
 	    "See HBASE-13985");
       }
-      discoverLoadQueue(queue, hfofDir, validateHFile);
-      // check whether there is invalid family name in HFiles to be bulkloaded
-      Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
-      ArrayList<String> familyNames = new ArrayList<String>(families.size());
-      for (HColumnDescriptor family : families) {
-        familyNames.add(family.getNameAsString());
-      }
-      ArrayList<String> unmatchedFamilies = new ArrayList<String>();
-      Iterator<LoadQueueItem> queueIter = queue.iterator();
-      while (queueIter.hasNext()) {
-        LoadQueueItem lqi = queueIter.next();
-        String familyNameInHFile = Bytes.toString(lqi.family);
-        if (!familyNames.contains(familyNameInHFile)) {
-          unmatchedFamilies.add(familyNameInHFile);
-        }
-      }
-      if (unmatchedFamilies.size() > 0) {
-        String msg =
-            "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
-                + unmatchedFamilies + "; valid family names of table "
-                + table.getName() + " are: " + familyNames;
-        LOG.error(msg);
-        throw new IOException(msg);
-      }
+      prepareHFileQueue(hfofDir, table, queue, validateHFile);
+
       int count = 0;
 
       if (queue.isEmpty()) {
@@ -397,7 +369,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
               + count + " with " + queue.size() + " files remaining to group or split");
         }
 
-        int maxRetries = getConf().getInt("hbase.bulkload.retries.number", 10);
+        int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
         maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
         if (maxRetries != 0 && count >= maxRetries) {
           throw new IOException("Retry attempted " + count +
@@ -447,6 +419,85 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
   }
 
   /**
+   * Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
+   * passed directory and validates whether the prepared queue has all the valid table column
+   * families in it.
+   * @param hfilesDir directory containing list of hfiles to be loaded into the table
+   * @param table table to which hfiles should be loaded
+   * @param queue queue which needs to be loaded into the table
+   * @throws IOException If any I/O or network error occurred
+   */
+  public void prepareHFileQueue(Path hfofDir, Table table, Deque<LoadQueueItem> queue,
+      boolean validateHFile) throws IOException {
+    discoverLoadQueue(queue, hfofDir, validateHFile);
+    validateFamiliesInHFiles(table, queue);
+  }
+
+  // Initialize a thread pool
+  private ExecutorService createExecutorService() {
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    builder.setNameFormat("LoadIncrementalHFiles-%1$d");
+    ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
+        new LinkedBlockingQueue<Runnable>(), builder.build());
+    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
+    return pool;
+  }
+
+  /**
+   * Checks whether there is any invalid family name in HFiles to be bulk loaded.
+   */
+  private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue)
+      throws IOException {
+    Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
+    List<String> familyNames = new ArrayList<String>(families.size());
+    for (HColumnDescriptor family : families) {
+      familyNames.add(family.getNameAsString());
+    }
+    List<String> unmatchedFamilies = new ArrayList<String>();
+    Iterator<LoadQueueItem> queueIter = queue.iterator();
+    while (queueIter.hasNext()) {
+      LoadQueueItem lqi = queueIter.next();
+      String familyNameInHFile = Bytes.toString(lqi.family);
+      if (!familyNames.contains(familyNameInHFile)) {
+        unmatchedFamilies.add(familyNameInHFile);
+      }
+    }
+    if (unmatchedFamilies.size() > 0) {
+      String msg =
+          "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
+              + unmatchedFamilies + "; valid family names of table " + table.getName() + " are: "
+              + familyNames;
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
+  }
+
+  /**
+   * Used by the replication sink to load the hfiles from the source cluster. It does the following,
+   * 1. {@link LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)} 2.
+   * {@link
+   * LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)}
+   * @param table Table to which these hfiles should be loaded to
+   * @param conn Connection to use
+   * @param queue {@link LoadQueueItem} has hfiles yet to be loaded
+   * @param startEndKeys starting and ending row keys of the region
+   */
+  public void loadHFileQueue(final Table table, final Connection conn, Deque<LoadQueueItem> queue,
+      Pair<byte[][], byte[][]> startEndKeys) throws IOException {
+    ExecutorService pool = null;
+    try {
+      pool = createExecutorService();
+      Multimap<ByteBuffer, LoadQueueItem> regionGroups =
+          groupOrSplitPhase(table, pool, queue, startEndKeys);
+      bulkLoadPhase(table, conn, pool, queue, regionGroups);
+    } finally {
+      if (pool != null) {
+        pool.shutdown();
+      }
+    }
+  }
+
+  /**
    * This takes the LQI's grouped by likely regions and attempts to bulk load
    * them.  Any failures are re-queued for another pass with the
    * groupOrSplitPhase.
@@ -592,10 +643,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
     String uniqueName = getUniqueName();
     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
+
     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
     Path topOut = new Path(tmpDir, uniqueName + ".top");
-    splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
-        botOut, topOut);
+    splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
 
     FileSystem fs = tmpDir.getFileSystem(getConf());
     fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
@@ -626,6 +677,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       final Pair<byte[][], byte[][]> startEndKeys)
       throws IOException {
     final Path hfilePath = item.hfilePath;
+    // fs is the source filesystem
+    if (fs == null) {
+      fs = hfilePath.getFileSystem(getConf());
+    }
     HFile.Reader hfr = HFile.createReader(fs, hfilePath,
         new CacheConfig(getConf()), getConf());
     final byte[] first, last;
@@ -712,7 +767,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
    * failure
    */
   protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
-      final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
+      final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
   throws IOException {
     final List<Pair<byte[], String>> famPaths =
       new ArrayList<Pair<byte[], String>>(lqis.size());
@@ -747,6 +802,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
           //in user directory
           if(secureClient != null && !success) {
             FileSystem targetFs = FileSystem.get(getConf());
+         // fs is the source filesystem
+            if(fs == null) {
+              fs = lqis.iterator().next().hfilePath.getFileSystem(getConf());
+            }
             // Check to see if the source and target filesystems are the same
             // If they are the same filesystem, we will try move the files back
             // because previously we moved them to the staging directory.
@@ -1000,4 +1059,17 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     System.exit(ret);
   }
 
+  /**
+   * Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
+   * used only when {@link SecureBulkLoadEndpoint} is configured in hbase.coprocessor.region.classes
+   * property. This directory is used as a temporary directory where all files are initially
+   * copied/moved from user given directory, set all the required file permissions and then from
+   * their it is finally loaded into a table. This should be set only when, one would like to manage
+   * the staging directory by itself. Otherwise this tool will handle this by itself.
+   * @param stagingDir staging directory path
+   */
+  public void setBulkToken(String stagingDir) {
+    this.bulkToken = stagingDir;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index d6a120b..91185af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -28,22 +28,23 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.UUID;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALKey;
 
 import com.google.protobuf.ServiceException;
 
@@ -51,15 +52,20 @@ import com.google.protobuf.ServiceException;
 public class ReplicationProtbufUtil {
   /**
    * A helper to replicate a list of WAL entries using admin protocol.
-   *
-   * @param admin
-   * @param entries
+   * @param admin Admin service
+   * @param entries Array of WAL entries to be replicated
+   * @param replicationClusterId Id which will uniquely identify source cluster FS client
+   *          configurations in the replication configuration directory
+   * @param sourceBaseNamespaceDir Path to source cluster base namespace directory
+   * @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
    * @throws java.io.IOException
    */
   public static void replicateWALEntry(final AdminService.BlockingInterface admin,
-      final Entry[] entries) throws IOException {
+      final Entry[] entries, String replicationClusterId, Path sourceBaseNamespaceDir,
+      Path sourceHFileArchiveDir) throws IOException {
     Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
-      buildReplicateWALEntryRequest(entries, null);
+        buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir,
+          sourceHFileArchiveDir);
     PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
     try {
       admin.replicateWALEntry(controller, p.getFirst());
@@ -77,19 +83,22 @@ public class ReplicationProtbufUtil {
    */
   public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
       buildReplicateWALEntryRequest(final Entry[] entries) {
-    return buildReplicateWALEntryRequest(entries, null);
+    return buildReplicateWALEntryRequest(entries, null, null, null, null);
   }
 
   /**
    * Create a new ReplicateWALEntryRequest from a list of WAL entries
-   *
    * @param entries the WAL entries to be replicated
    * @param encodedRegionName alternative region name to use if not null
-   * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values
-   * found.
+   * @param replicationClusterId Id which will uniquely identify source cluster FS client
+   *          configurations in the replication configuration directory
+   * @param sourceBaseNamespaceDir Path to source cluster base namespace directory
+   * @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
+   * @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values found.
    */
   public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
-      buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName) {
+      buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName,
+          String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir) {
     // Accumulate all the Cells seen in here.
     List<List<? extends Cell>> allCells = new ArrayList<List<? extends Cell>>(entries.length);
     int size = 0;
@@ -146,6 +155,17 @@ public class ReplicationProtbufUtil {
       entryBuilder.setAssociatedCellCount(cells.size());
       builder.addEntry(entryBuilder.build());
     }
+
+    if (replicationClusterId != null) {
+      builder.setReplicationClusterId(replicationClusterId);
+    }
+    if (sourceBaseNamespaceDir != null) {
+      builder.setSourceBaseNamespaceDirPath(sourceBaseNamespaceDir.toString());
+    }
+    if (sourceHFileArchiveDir != null) {
+      builder.setSourceHFileArchiveDirPath(sourceHFileArchiveDir.toString());
+    }
+
     return new Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>(builder.build(),
       getCellScanner(allCells, size));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index d94e11c..0c9b0e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1800,7 +1800,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         List<WALEntry> entries = request.getEntryList();
         CellScanner cellScanner = ((PayloadCarryingRpcController)controller).cellScanner();
         regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries(entries, cellScanner);
-        regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner);
+        regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner,
+          request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(),
+          request.getSourceHFileArchiveDirPath());
         regionServer.getRegionServerCoprocessorHost().postReplicateLogEntries(entries, cellScanner);
         return ReplicateWALEntryResponse.newBuilder().build();
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java
index 5f96bf7..836d3aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSinkService.java
@@ -36,7 +36,13 @@ public interface ReplicationSinkService extends ReplicationService {
    * Carry on the list of log entries down to the sink
    * @param entries list of WALEntries to replicate
    * @param cells Cells that the WALEntries refer to (if cells is non-null)
+   * @param replicationClusterId Id which will uniquely identify source cluster FS client
+   *          configurations in the replication configuration directory
+   * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
+   *          directory required for replicating hfiles
+   * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
    * @throws IOException
    */
-  void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException;
+  void replicateLogEntries(List<WALEntry> entries, CellScanner cells, String replicationClusterId,
+      String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath) throws IOException;
 }


[08/26] hbase git commit: HBASE-14941 locate_region shell command

Posted by sy...@apache.org.
HBASE-14941 locate_region shell command


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

Branch: refs/heads/hbase-12439
Commit: 6f8d5e86cee2554ebbe6b4d34d828deff04aa894
Parents: 26ac60b
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Dec 10 13:22:48 2015 -0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Dec 10 13:22:48 2015 -0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/admin.rb        | 26 ++++++++----
 hbase-shell/src/main/ruby/shell.rb              |  1 +
 .../main/ruby/shell/commands/locate_region.rb   | 44 ++++++++++++++++++++
 3 files changed, 64 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6f8d5e86/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index bd24065..9a32b0e 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -132,6 +132,15 @@ module Hbase
       end
     end
 
+    def locate_region(table_name, row_key)
+      locator = @connection.getRegionLocator(TableName.valueOf(table_name))
+      begin
+        return locator.getRegionLocation(Bytes.toBytesBinary(row_key))
+      ensure
+        locator.close()
+      end
+    end
+
     #----------------------------------------------------------------------------------------------
     # Requests a cluster balance
     # Returns true if balancer ran
@@ -469,10 +478,13 @@ module Hbase
     def truncate_preserve(table_name, conf = @conf)
       h_table = @connection.getTable(TableName.valueOf(table_name))
       locator = @connection.getRegionLocator(TableName.valueOf(table_name))
-      splits = locator.getAllRegionLocations().
-          map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}.
-          delete_if{|k| k == ""}.to_java :String
-      locator.close()
+      begin
+        splits = locator.getAllRegionLocations().
+            map{|i| Bytes.toString(i.getRegionInfo().getStartKey)}.
+            delete_if{|k| k == ""}.to_java :String
+      ensure
+        locator.close()
+      end
 
       table_description = @admin.getTableDescriptor(TableName.valueOf(table_name))
       yield 'Disabling table...' if block_given?
@@ -707,7 +719,7 @@ module Hbase
         end
       elsif format == "replication"
         #check whether replication is enabled or not
-        if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_KEY, 
+        if (!@admin.getConfiguration().getBoolean(org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_KEY,
           org.apache.hadoop.hbase.HConstants::REPLICATION_ENABLE_DEFAULT))
           puts("Please enable replication first.")
         else
@@ -719,7 +731,7 @@ module Hbase
             rSourceString = "       SOURCE:"
             rLoadSink = sl.getReplicationLoadSink()
             rSinkString << " AgeOfLastAppliedOp=" + rLoadSink.getAgeOfLastAppliedOp().to_s
-            rSinkString << ", TimeStampsOfLastAppliedOp=" + 
+            rSinkString << ", TimeStampsOfLastAppliedOp=" +
 			    (java.util.Date.new(rLoadSink.getTimeStampsOfLastAppliedOp())).toString()
             rLoadSourceList = sl.getReplicationLoadSourceList()
             index = 0
@@ -728,7 +740,7 @@ module Hbase
               rSourceString << " PeerID=" + rLoadSource.getPeerID()
               rSourceString << ", AgeOfLastShippedOp=" + rLoadSource.getAgeOfLastShippedOp().to_s
               rSourceString << ", SizeOfLogQueue=" + rLoadSource.getSizeOfLogQueue().to_s
-              rSourceString << ", TimeStampsOfLastShippedOp=" + 
+              rSourceString << ", TimeStampsOfLastShippedOp=" +
 			      (java.util.Date.new(rLoadSource.getTimeStampOfLastShippedOp())).toString()
               rSourceString << ", Replication Lag=" + rLoadSource.getReplicationLag().to_s
               index = index + 1

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f8d5e86/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 587669d..0ecd3d7 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -267,6 +267,7 @@ Shell.load_command_group(
     alter_status
     alter_async
     get_table
+    locate_region
   ],
   :aliases => {
     'describe' => ['desc']

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f8d5e86/hbase-shell/src/main/ruby/shell/commands/locate_region.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/locate_region.rb b/hbase-shell/src/main/ruby/shell/commands/locate_region.rb
new file mode 100644
index 0000000..b1e8c7b
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/locate_region.rb
@@ -0,0 +1,44 @@
+#
+# Copyright The Apache Software Foundation
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class LocateRegion < Command
+      def help
+        return <<-EOF
+Locate the region given a table name and a row-key
+
+  hbase> locate_region 'tableName', 'key0'
+EOF
+      end
+
+      def command(table, row_key)
+        now = Time.now
+
+        region_location = admin.locate_region(table, row_key)
+        hri = region_location.getRegionInfo()
+
+        formatter.header([ "HOST", "REGION" ])
+        formatter.row([region_location.getHostnamePort(), hri.toString()])
+        formatter.footer(now, 1)
+      end
+    end
+  end
+end


[26/26] hbase git commit: HBASE-14967 Remove max-width constraint from website skin

Posted by sy...@apache.org.
HBASE-14967 Remove max-width constraint from website skin


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

Branch: refs/heads/hbase-12439
Commit: 64c55b4893ba134f2b2ba849c23027c891638b5c
Parents: ef92a6a
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Fri Dec 11 10:10:50 2015 -0800
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Tue Dec 15 08:56:35 2015 -0800

----------------------------------------------------------------------
 src/main/site/resources/css/site.css | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/64c55b48/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
index 8e7acb6..3f42f5a 100644
--- a/src/main/site/resources/css/site.css
+++ b/src/main/site/resources/css/site.css
@@ -35,7 +35,6 @@ div#breadcrumbs,
 div#bodyColumn,
 footer {
   width: initial;
-  max-width: 1000px;
   padding-left: 20px;
   padding-right: 20px;
   clear: both;


[16/26] hbase git commit: HBASE-14960 Fallback to using default RPCControllerFactory if class cannot be loaded

Posted by sy...@apache.org.
HBASE-14960 Fallback to using default RPCControllerFactory if class cannot be loaded


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

Branch: refs/heads/hbase-12439
Commit: cff664c5e286bebaddd93665680fb148783b8e7a
Parents: abb2e95
Author: Enis Soztutar <en...@apache.org>
Authored: Thu Dec 10 19:11:57 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Thu Dec 10 19:11:57 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/RpcControllerFactory.java  | 26 ++++++++++++++++++--
 .../hbase/client/TestRpcControllerFactory.java  | 17 ++++++++++++-
 2 files changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cff664c5/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
index f8ab23f..faeca8d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcControllerFactory.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.ipc;
 
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
@@ -30,7 +32,13 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
  */
 @InterfaceAudience.Private
 public class RpcControllerFactory {
+  private static final Log LOG = LogFactory.getLog(RpcControllerFactory.class);
 
+  /**
+   * Custom RPC Controller factory allows frameworks to change the RPC controller. If the configured
+   * controller cannot be found in the classpath or loaded, we fall back to the default RPC
+   * controller factory.
+   */
   public static final String CUSTOM_CONTROLLER_CONF_KEY = "hbase.rpc.controllerfactory.class";
   protected final Configuration conf;
 
@@ -55,7 +63,21 @@ public class RpcControllerFactory {
     String rpcControllerFactoryClazz =
         configuration.get(CUSTOM_CONTROLLER_CONF_KEY,
           RpcControllerFactory.class.getName());
-    return ReflectionUtils.instantiateWithCustomCtor(rpcControllerFactoryClazz,
-      new Class[] { Configuration.class }, new Object[] { configuration });
+    try {
+      return ReflectionUtils.instantiateWithCustomCtor(rpcControllerFactoryClazz,
+        new Class[] { Configuration.class }, new Object[] { configuration });
+    } catch (UnsupportedOperationException | NoClassDefFoundError ex) {
+      // HBASE-14960: In case the RPCController is in a non-HBase jar (Phoenix), but the application
+      // is a pure HBase application, we want to fallback to the default one.
+      String msg = "Cannot load configured \"" + CUSTOM_CONTROLLER_CONF_KEY + "\" ("
+          + rpcControllerFactoryClazz + ") from hbase-site.xml, falling back to use "
+          + "default RpcControllerFactory";
+      if (LOG.isDebugEnabled()) {
+        LOG.warn(msg, ex); // if DEBUG enabled, we want the exception, but still log in WARN level
+      } else {
+        LOG.warn(msg);
+      }
+      return new RpcControllerFactory(configuration);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cff664c5/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
index c087135..656dedc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.io.IOException;
 import java.util.List;
@@ -54,14 +55,17 @@ public class TestRpcControllerFactory {
       super(conf);
     }
 
+    @Override
     public PayloadCarryingRpcController newController() {
       return new CountingRpcController(super.newController());
     }
 
+    @Override
     public PayloadCarryingRpcController newController(final CellScanner cellScanner) {
       return new CountingRpcController(super.newController(cellScanner));
     }
 
+    @Override
     public PayloadCarryingRpcController newController(final List<CellScannable> cellIterables) {
       return new CountingRpcController(super.newController(cellIterables));
     }
@@ -103,7 +107,7 @@ public class TestRpcControllerFactory {
     Configuration conf = UTIL.getConfiguration();
     conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       ProtobufCoprocessorService.class.getName());
-    
+
     UTIL.startMiniCluster();
   }
 
@@ -202,4 +206,15 @@ public class TestRpcControllerFactory {
     assertEquals(0, CountingRpcController.INT_PRIORITY.get());
     return counter + 1;
   }
+
+  @Test
+  public void testFallbackToDefaultRpcControllerFactory() {
+    Configuration conf = new Configuration(UTIL.getConfiguration());
+    conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY, "foo.bar.Baz");
+
+    // Should not fail
+    RpcControllerFactory factory = RpcControllerFactory.instantiate(conf);
+    assertNotNull(factory);
+    assertEquals(factory.getClass(), RpcControllerFactory.class);
+  }
 }
\ No newline at end of file


[05/26] hbase git commit: HBASE-13153 Bulk Loaded HFile Replication (Ashish Singhi)

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index a8cffba..9ff4b2d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -45,8 +45,10 @@ 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.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
@@ -225,8 +227,16 @@ public class ReplicationSourceManager implements ReplicationListener {
    * old region server wal queues
    */
   protected void init() throws IOException, ReplicationException {
+    boolean replicationForBulkLoadDataEnabled =
+        conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+          HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
     for (String id : this.replicationPeers.getPeerIds()) {
       addSource(id);
+      if (replicationForBulkLoadDataEnabled) {
+        // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
+        // when a peer was added before replication for bulk loaded data was enabled.
+        this.replicationQueues.addPeerToHFileRefs(id);
+      }
     }
     List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
     if (currentReplicators == null || currentReplicators.size() == 0) {
@@ -733,4 +743,15 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
     return stats.toString();
   }
+
+  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+      throws ReplicationException {
+    for (ReplicationSourceInterface source : this.sources) {
+      source.addHFileRefs(tableName, family, files);
+    }
+  }
+
+  public void cleanUpHFileRefs(String peerId, List<String> files) {
+    this.replicationQueues.removeHFileRefs(peerId, files);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SourceFSConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SourceFSConfigurationProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SourceFSConfigurationProvider.java
new file mode 100644
index 0000000..8271115
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SourceFSConfigurationProvider.java
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Interface that defines how a region server in peer cluster will get source cluster file system
+ * configurations. User can configure their custom implementation implementing this interface by
+ * setting the value of their custom implementation's fully qualified class name to
+ * hbase.replication.source.fs.conf.provider property in RegionServer configuration. Default is
+ * {@link DefaultSourceFSConfigurationProvider}
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
+public interface SourceFSConfigurationProvider {
+
+  /**
+   * Returns the source cluster file system configuration for the given source cluster replication
+   * ID.
+   * @param sinkConf sink cluster configuration
+   * @param replicationClusterId unique ID which identifies the source cluster
+   * @return source cluster file system configuration
+   * @throws IOException for invalid directory or for a bad disk.
+   */
+  public Configuration getConf(Configuration sinkConf, String replicationClusterId)
+      throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
index becc9f3..3541ade 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java
@@ -217,7 +217,7 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
     for(ClientProtos.BulkLoadHFileRequest.FamilyPath el : request.getFamilyPathList()) {
       familyPaths.add(new Pair(el.getFamily().toByteArray(),el.getPath()));
     }
-    
+
     Token userToken = null;
     if (userProvider.isHadoopSecurityEnabled()) {
       userToken = new Token(request.getFsToken().getIdentifier().toByteArray(), request.getFsToken()
@@ -375,6 +375,14 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
     public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
       Path p = new Path(srcPath);
       Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
+
+      // In case of Replication for bulk load files, hfiles are already copied in staging directory
+      if (p.equals(stageP)) {
+        LOG.debug(p.getName()
+            + " is already available in staging directory. Skipping copy or rename.");
+        return stageP.toString();
+      }
+
       if (srcFs == null) {
         srcFs = FileSystem.get(p.toUri(), conf);
       }
@@ -414,6 +422,14 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
       Path p = new Path(srcPath);
       Path stageP = new Path(stagingDir,
           new Path(Bytes.toString(family), p.getName()));
+
+      // In case of Replication for bulk load files, hfiles are not renamed by end point during
+      // prepare stage, so no need of rename here again
+      if (p.equals(stageP)) {
+        LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
+        return;
+      }
+
       LOG.debug("Moving " + stageP + " back to " + p);
       if(!fs.rename(stageP, p))
         throw new IOException("Failed to move HFile: " + stageP + " to " + p);

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
new file mode 100644
index 0000000..87db386
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -0,0 +1,264 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ChoreService;
+import org.apache.hadoop.hbase.CoordinatedStateManager;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category({ MasterTests.class, SmallTests.class })
+public class TestReplicationHFileCleaner {
+  private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Server server;
+  private static ReplicationQueues rq;
+  private static ReplicationPeers rp;
+  private static final String peerId = "TestReplicationHFileCleaner";
+  private static Configuration conf = TEST_UTIL.getConfiguration();
+  static FileSystem fs = null;
+  Path root;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+    server = new DummyServer();
+    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+    Replication.decorateMasterConfiguration(conf);
+    rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
+    rp.init();
+
+    rq = ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
+    rq.init(server.getServerName().toString());
+    try {
+      fs = FileSystem.get(conf);
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+    }
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Before
+  public void setup() throws ReplicationException, IOException {
+    root = TEST_UTIL.getDataTestDirOnTestFS();
+    rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()), null);
+  }
+
+  @After
+  public void cleanup() throws ReplicationException {
+    try {
+      fs.delete(root, true);
+    } catch (IOException e) {
+      LOG.warn("Failed to delete files recursively from path " + root);
+    }
+    rp.removePeer(peerId);
+  }
+
+  @Test
+  public void testIsFileDeletable() throws IOException, ReplicationException {
+    // 1. Create a file
+    Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
+    fs.createNewFile(file);
+    // 2. Assert file is successfully created
+    assertTrue("Test file not created!", fs.exists(file));
+    ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
+    cleaner.setConf(conf);
+    // 3. Assert that file as is should be deletable
+    assertTrue("Cleaner should allow to delete this file as there is no hfile reference node "
+        + "for it in the queue.",
+      cleaner.isFileDeletable(fs.getFileStatus(file)));
+
+    List<String> files = new ArrayList<String>(1);
+    files.add(file.getName());
+    // 4. Add the file to hfile-refs queue
+    rq.addHFileRefs(peerId, files);
+    // 5. Assert file should not be deletable
+    assertFalse("Cleaner should not allow to delete this file as there is a hfile reference node "
+        + "for it in the queue.",
+      cleaner.isFileDeletable(fs.getFileStatus(file)));
+  }
+
+  @Test
+  public void testGetDeletableFiles() throws Exception {
+    // 1. Create two files and assert that they do not exist
+    Path notDeletablefile = new Path(root, "testGetDeletableFiles_1");
+    fs.createNewFile(notDeletablefile);
+    assertTrue("Test file not created!", fs.exists(notDeletablefile));
+    Path deletablefile = new Path(root, "testGetDeletableFiles_2");
+    fs.createNewFile(deletablefile);
+    assertTrue("Test file not created!", fs.exists(deletablefile));
+
+    List<FileStatus> files = new ArrayList<FileStatus>(2);
+    FileStatus f = new FileStatus();
+    f.setPath(deletablefile);
+    files.add(f);
+    f = new FileStatus();
+    f.setPath(notDeletablefile);
+    files.add(f);
+
+    List<String> hfiles = new ArrayList<>(1);
+    hfiles.add(notDeletablefile.getName());
+    // 2. Add one file to hfile-refs queue
+    rq.addHFileRefs(peerId, hfiles);
+
+    ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
+    cleaner.setConf(conf);
+    Iterator<FileStatus> deletableFilesIterator = cleaner.getDeletableFiles(files).iterator();
+    int i = 0;
+    while (deletableFilesIterator.hasNext() && i < 2) {
+      i++;
+    }
+    // 5. Assert one file should not be deletable and it is present in the list returned
+    if (i > 2) {
+      fail("File " + notDeletablefile
+          + " should not be deletable as its hfile reference node is not added.");
+    }
+    assertTrue(deletableFilesIterator.next().getPath().equals(deletablefile));
+  }
+
+  /*
+   * Test for HBASE-14621. This test will not assert directly anything. Without the fix the test
+   * will end up in a infinite loop, so it will timeout.
+   */
+  @Test(timeout = 15000)
+  public void testForDifferntHFileRefsZnodeVersion() throws Exception {
+    // 1. Create a file
+    Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion");
+    fs.createNewFile(file);
+    // 2. Assert file is successfully created
+    assertTrue("Test file not created!", fs.exists(file));
+    ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
+    cleaner.setConf(conf);
+
+    ReplicationQueuesClient replicationQueuesClient = Mockito.mock(ReplicationQueuesClient.class);
+    //Return different znode version for each call
+    Mockito.when(replicationQueuesClient.getHFileRefsNodeChangeVersion()).thenReturn(1, 2);
+
+    Class<? extends ReplicationHFileCleaner> cleanerClass = cleaner.getClass();
+    Field rqc = cleanerClass.getDeclaredField("rqc");
+    rqc.setAccessible(true);
+    rqc.set(cleaner, replicationQueuesClient);
+
+    cleaner.isFileDeletable(fs.getFileStatus(file));
+  }
+
+  static class DummyServer implements Server {
+
+    @Override
+    public Configuration getConfiguration() {
+      return TEST_UTIL.getConfiguration();
+    }
+
+    @Override
+    public ZooKeeperWatcher getZooKeeper() {
+      try {
+        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      return null;
+    }
+
+    @Override
+    public CoordinatedStateManager getCoordinatedStateManager() {
+      return null;
+    }
+
+    @Override
+    public ClusterConnection getConnection() {
+      return null;
+    }
+
+    @Override
+    public MetaTableLocator getMetaTableLocator() {
+      return null;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return ServerName.valueOf("regionserver,60020,000000");
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public void stop(String why) {
+    }
+
+    @Override
+    public boolean isStopped() {
+      return false;
+    }
+
+    @Override
+    public ChoreService getChoreService() {
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index f463f76..abe484e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -19,12 +19,14 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
@@ -89,4 +91,10 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   public String getStats() {
     return "";
   }
+
+  @Override
+  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+      throws ReplicationException {
+    return;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index 455a790..e919c24 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -19,15 +19,21 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+
 import org.apache.commons.logging.Log;
 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.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -35,7 +41,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+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.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
@@ -48,12 +56,17 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.junit.After;
@@ -79,6 +92,7 @@ public class TestMasterReplication {
 
   private static final TableName tableName = TableName.valueOf("test");
   private static final byte[] famName = Bytes.toBytes("f");
+  private static final byte[] famName1 = Bytes.toBytes("f1");
   private static final byte[] row = Bytes.toBytes("row");
   private static final byte[] row1 = Bytes.toBytes("row1");
   private static final byte[] row2 = Bytes.toBytes("row2");
@@ -103,7 +117,11 @@ public class TestMasterReplication {
     baseConfiguration.setInt("hbase.regionserver.maxlogs", 10);
     baseConfiguration.setLong("hbase.master.logcleaner.ttl", 10);
     baseConfiguration.setBoolean(HConstants.REPLICATION_ENABLE_KEY,
-        HConstants.REPLICATION_ENABLE_DEFAULT);
+      HConstants.REPLICATION_ENABLE_DEFAULT);
+    baseConfiguration.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+    baseConfiguration.set("hbase.replication.source.fs.conf.provider",
+      TestSourceFSConfigurationProvider.class.getCanonicalName());
+    baseConfiguration.set(HConstants.REPLICATION_CLUSTER_ID, "12345");
     baseConfiguration.setBoolean("dfs.support.append", true);
     baseConfiguration.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
     baseConfiguration.setStrings(
@@ -114,6 +132,9 @@ public class TestMasterReplication {
     HColumnDescriptor fam = new HColumnDescriptor(famName);
     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
     table.addFamily(fam);
+    fam = new HColumnDescriptor(famName1);
+    fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
+    table.addFamily(fam);
     fam = new HColumnDescriptor(noRepfamName);
     table.addFamily(fam);
   }
@@ -130,14 +151,7 @@ public class TestMasterReplication {
     int numClusters = 2;
     Table[] htables = null;
     try {
-      startMiniClusters(numClusters);
-      createTableOnClusters(table);
-
-      htables = getHTablesOnClusters(tableName);
-
-      // Test the replication scenarios of 0 -> 1 -> 0
-      addPeer("1", 0, 1);
-      addPeer("1", 1, 0);
+      htables = setUpClusterTablesAndPeers(numClusters);
 
       int[] expectedCounts = new int[] { 2, 2 };
 
@@ -157,12 +171,64 @@ public class TestMasterReplication {
   }
 
   /**
-   * Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and
-   * deleting rows to a table in each clusters and ensuring that the each of
-   * these clusters get the appropriate mutations. It also tests the grouping
-   * scenario where a cluster needs to replicate the edits originating from
-   * itself and also the edits that it received using replication from a
-   * different cluster. The scenario is explained in HBASE-9158
+   * It tests the replication scenario involving 0 -> 1 -> 0. It does it by bulk loading a set of
+   * HFiles to a table in each cluster, checking if it's replicated.
+   */
+  @Test(timeout = 300000)
+  public void testHFileCyclicReplication() throws Exception {
+    LOG.info("testHFileCyclicReplication");
+    int numClusters = 2;
+    Table[] htables = null;
+    try {
+      htables = setUpClusterTablesAndPeers(numClusters);
+
+      // Load 100 rows for each hfile range in cluster '0' and validate whether its been replicated
+      // to cluster '1'.
+      byte[][][] hfileRanges =
+          new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
+              new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("fff") }, };
+      int numOfRows = 100;
+      int[] expectedCounts =
+          new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
+
+      loadAndValidateHFileReplication("testHFileCyclicReplication_01", 0, new int[] { 1 }, row,
+        famName, htables, hfileRanges, numOfRows, expectedCounts, true);
+
+      // Load 200 rows for each hfile range in cluster '1' and validate whether its been replicated
+      // to cluster '0'.
+      hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("gggg"), Bytes.toBytes("iiii") },
+          new byte[][] { Bytes.toBytes("jjj"), Bytes.toBytes("lll") }, };
+      numOfRows = 200;
+      int[] newExpectedCounts = new int[] { hfileRanges.length * numOfRows + expectedCounts[0],
+          hfileRanges.length * numOfRows + expectedCounts[1] };
+
+      loadAndValidateHFileReplication("testHFileCyclicReplication_10", 1, new int[] { 0 }, row,
+        famName, htables, hfileRanges, numOfRows, newExpectedCounts, true);
+
+    } finally {
+      close(htables);
+      shutDownMiniClusters();
+    }
+  }
+
+  private Table[] setUpClusterTablesAndPeers(int numClusters) throws Exception {
+    Table[] htables;
+    startMiniClusters(numClusters);
+    createTableOnClusters(table);
+
+    htables = getHTablesOnClusters(tableName);
+    // Test the replication scenarios of 0 -> 1 -> 0
+    addPeer("1", 0, 1);
+    addPeer("1", 1, 0);
+    return htables;
+  }
+
+  /**
+   * Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and deleting rows to a
+   * table in each clusters and ensuring that the each of these clusters get the appropriate
+   * mutations. It also tests the grouping scenario where a cluster needs to replicate the edits
+   * originating from itself and also the edits that it received using replication from a different
+   * cluster. The scenario is explained in HBASE-9158
    */
   @Test(timeout = 300000)
   public void testCyclicReplication2() throws Exception {
@@ -213,6 +279,119 @@ public class TestMasterReplication {
   }
 
   /**
+   * It tests the multi slave hfile replication scenario involving 0 -> 1, 2. It does it by bulk
+   * loading a set of HFiles to a table in master cluster, checking if it's replicated in its peers.
+   */
+  @Test(timeout = 300000)
+  public void testHFileMultiSlaveReplication() throws Exception {
+    LOG.info("testHFileMultiSlaveReplication");
+    int numClusters = 3;
+    Table[] htables = null;
+    try {
+      startMiniClusters(numClusters);
+      createTableOnClusters(table);
+
+      // Add a slave, 0 -> 1
+      addPeer("1", 0, 1);
+
+      htables = getHTablesOnClusters(tableName);
+
+      // Load 100 rows for each hfile range in cluster '0' and validate whether its been replicated
+      // to cluster '1'.
+      byte[][][] hfileRanges =
+          new byte[][][] { new byte[][] { Bytes.toBytes("mmmm"), Bytes.toBytes("oooo") },
+              new byte[][] { Bytes.toBytes("ppp"), Bytes.toBytes("rrr") }, };
+      int numOfRows = 100;
+
+      int[] expectedCounts =
+          new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
+
+      loadAndValidateHFileReplication("testHFileCyclicReplication_0", 0, new int[] { 1 }, row,
+        famName, htables, hfileRanges, numOfRows, expectedCounts, true);
+
+      // Validate data is not replicated to cluster '2'.
+      assertEquals(0, utilities[2].countRows(htables[2]));
+
+      rollWALAndWait(utilities[0], htables[0].getName(), row);
+
+      // Add one more slave, 0 -> 2
+      addPeer("2", 0, 2);
+
+      // Load 200 rows for each hfile range in cluster '0' and validate whether its been replicated
+      // to cluster '1' and '2'. Previous data should be replicated to cluster '2'.
+      hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("ssss"), Bytes.toBytes("uuuu") },
+          new byte[][] { Bytes.toBytes("vvv"), Bytes.toBytes("xxx") }, };
+      numOfRows = 200;
+
+      int[] newExpectedCounts = new int[] { hfileRanges.length * numOfRows + expectedCounts[0],
+          hfileRanges.length * numOfRows + expectedCounts[1], hfileRanges.length * numOfRows };
+
+      loadAndValidateHFileReplication("testHFileCyclicReplication_1", 0, new int[] { 1, 2 }, row,
+        famName, htables, hfileRanges, numOfRows, newExpectedCounts, true);
+
+    } finally {
+      close(htables);
+      shutDownMiniClusters();
+    }
+  }
+  
+  /**
+   * It tests the bulk loaded hfile replication scenario to only explicitly specified table column
+   * families. It does it by bulk loading a set of HFiles belonging to both the CFs of table and set
+   * only one CF data to replicate.
+   */
+  @Test(timeout = 300000)
+  public void testHFileReplicationForConfiguredTableCfs() throws Exception {
+    LOG.info("testHFileReplicationForConfiguredTableCfs");
+    int numClusters = 2;
+    Table[] htables = null;
+    try {
+      startMiniClusters(numClusters);
+      createTableOnClusters(table);
+
+      htables = getHTablesOnClusters(tableName);
+      // Test the replication scenarios only 'f' is configured for table data replication not 'f1'
+      addPeer("1", 0, 1, tableName.getNameAsString() + ":" + Bytes.toString(famName));
+
+      // Load 100 rows for each hfile range in cluster '0' for table CF 'f'
+      byte[][][] hfileRanges =
+          new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
+              new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("fff") }, };
+      int numOfRows = 100;
+      int[] expectedCounts =
+          new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
+
+      loadAndValidateHFileReplication("load_f", 0, new int[] { 1 }, row, famName, htables,
+        hfileRanges, numOfRows, expectedCounts, true);
+
+      // Load 100 rows for each hfile range in cluster '0' for table CF 'f1'
+      hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("gggg"), Bytes.toBytes("iiii") },
+          new byte[][] { Bytes.toBytes("jjj"), Bytes.toBytes("lll") }, };
+      numOfRows = 100;
+
+      int[] newExpectedCounts =
+          new int[] { hfileRanges.length * numOfRows + expectedCounts[0], expectedCounts[1] };
+
+      loadAndValidateHFileReplication("load_f1", 0, new int[] { 1 }, row, famName1, htables,
+        hfileRanges, numOfRows, newExpectedCounts, false);
+
+      // Validate data replication for CF 'f1'
+
+      // Source cluster table should contain data for the families
+      wait(0, htables[0], hfileRanges.length * numOfRows + expectedCounts[0]);
+
+      // Sleep for enough time so that the data is still not replicated for the CF which is not
+      // configured for replication
+      Thread.sleep((NB_RETRIES / 2) * SLEEP_TIME);
+      // Peer cluster should have only configured CF data
+      wait(1, htables[1], expectedCounts[1]);
+    } finally {
+      close(htables);
+      shutDownMiniClusters();
+    }
+  }
+
+  /**
    * Tests cyclic replication scenario of 0 -> 1 -> 2 -> 1.
    */
   @Test(timeout = 300000)
@@ -328,6 +507,17 @@ public class TestMasterReplication {
       close(replicationAdmin);
     }
   }
+  
+  private void addPeer(String id, int masterClusterNumber, int slaveClusterNumber, String tableCfs)
+      throws Exception {
+    ReplicationAdmin replicationAdmin = null;
+    try {
+      replicationAdmin = new ReplicationAdmin(configurations[masterClusterNumber]);
+      replicationAdmin.addPeer(id, utilities[slaveClusterNumber].getClusterKey(), tableCfs);
+    } finally {
+      close(replicationAdmin);
+    }
+  }
 
   private void disablePeer(String id, int masterClusterNumber) throws Exception {
     ReplicationAdmin replicationAdmin = null;
@@ -405,8 +595,56 @@ public class TestMasterReplication {
     wait(row, target, false);
   }
 
-  private void wait(byte[] row, Table target, boolean isDeleted)
-      throws Exception {
+  private void loadAndValidateHFileReplication(String testName, int masterNumber,
+      int[] slaveNumbers, byte[] row, byte[] fam, Table[] tables, byte[][][] hfileRanges,
+      int numOfRows, int[] expectedCounts, boolean toValidate) throws Exception {
+    HBaseTestingUtility util = utilities[masterNumber];
+
+    Path dir = util.getDataTestDirOnTestFS(testName);
+    FileSystem fs = util.getTestFileSystem();
+    dir = dir.makeQualified(fs);
+    Path familyDir = new Path(dir, Bytes.toString(fam));
+
+    int hfileIdx = 0;
+    for (byte[][] range : hfileRanges) {
+      byte[] from = range[0];
+      byte[] to = range[1];
+      HFileTestUtil.createHFile(util.getConfiguration(), fs,
+        new Path(familyDir, "hfile_" + hfileIdx++), fam, row, from, to, numOfRows);
+    }
+
+    Table source = tables[masterNumber];
+    final TableName tableName = source.getName();
+    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
+    String[] args = { dir.toString(), tableName.toString() };
+    loader.run(args);
+
+    if (toValidate) {
+      for (int slaveClusterNumber : slaveNumbers) {
+        wait(slaveClusterNumber, tables[slaveClusterNumber], expectedCounts[slaveClusterNumber]);
+      }
+    }
+  }
+
+  private void wait(int slaveNumber, Table target, int expectedCount)
+      throws IOException, InterruptedException {
+    int count = 0;
+    for (int i = 0; i < NB_RETRIES; i++) {
+      if (i == NB_RETRIES - 1) {
+        fail("Waited too much time for bulkloaded data replication. Current count=" + count
+            + ", expected count=" + expectedCount);
+      }
+      count = utilities[slaveNumber].countRows(target);
+      if (count != expectedCount) {
+        LOG.info("Waiting more time for bulkloaded data replication.");
+        Thread.sleep(SLEEP_TIME);
+      } else {
+        break;
+      }
+    }
+  }
+
+  private void wait(byte[] row, Table target, boolean isDeleted) throws Exception {
     Get get = new Get(row);
     for (int i = 0; i < NB_RETRIES; i++) {
       if (i == NB_RETRIES - 1) {
@@ -430,6 +668,47 @@ public class TestMasterReplication {
     }
   }
 
+  private void rollWALAndWait(final HBaseTestingUtility utility, final TableName table,
+      final byte[] row) throws IOException {
+    final Admin admin = utility.getHBaseAdmin();
+    final MiniHBaseCluster cluster = utility.getMiniHBaseCluster();
+
+    // find the region that corresponds to the given row.
+    HRegion region = null;
+    for (HRegion candidate : cluster.getRegions(table)) {
+      if (HRegion.rowIsInRange(candidate.getRegionInfo(), row)) {
+        region = candidate;
+        break;
+      }
+    }
+    assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region);
+
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    // listen for successful log rolls
+    final WALActionsListener listener = new WALActionsListener.Base() {
+          @Override
+          public void postLogRoll(final Path oldPath, final Path newPath) throws IOException {
+            latch.countDown();
+          }
+        };
+    region.getWAL().registerWALActionsListener(listener);
+
+    // request a roll
+    admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(),
+      region.getRegionInfo().getRegionName()));
+
+    // wait
+    try {
+      latch.await();
+    } catch (InterruptedException exception) {
+      LOG.warn("Interrupted while waiting for the wal of '" + region + "' to roll. If later " +
+          "replication tests fail, it's probably because we should still be waiting.");
+      Thread.currentThread().interrupt();
+    }
+    region.getWAL().unregisterWALActionsListener(listener);
+  }
+
   /**
    * Use a coprocessor to count puts and deletes. as KVs would be replicated back with the same
    * timestamp there is otherwise no way to count them.

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 4823597..47d2880 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -658,7 +658,8 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     HRegionInfo hri = new HRegionInfo(htable1.getName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit);
+    Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit,
+      htable1.getConfiguration(), null);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 696c130..41c3240 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.*;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.SortedMap;
 import java.util.SortedSet;
@@ -160,6 +161,62 @@ public abstract class TestReplicationStateBasic {
   }
 
   @Test
+  public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
+    rp.init();
+    rq1.init(server1);
+    rqc.init();
+
+    List<String> files1 = new ArrayList<String>(3);
+    files1.add("file_1");
+    files1.add("file_2");
+    files1.add("file_3");
+    assertNull(rqc.getReplicableHFiles(ID_ONE));
+    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rq1.addHFileRefs(ID_ONE, files1);
+    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
+    List<String> files2 = new ArrayList<>(files1);
+    String removedString = files2.remove(0);
+    rq1.removeHFileRefs(ID_ONE, files2);
+    assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
+    files2 = new ArrayList<>(1);
+    files2.add(removedString);
+    rq1.removeHFileRefs(ID_ONE, files2);
+    assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
+    rp.removePeer(ID_ONE);
+  }
+
+  @Test
+  public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
+    rq1.init(server1);
+    rqc.init();
+
+    rp.init();
+    rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
+    rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
+
+    List<String> files1 = new ArrayList<String>(3);
+    files1.add("file_1");
+    files1.add("file_2");
+    files1.add("file_3");
+    rq1.addHFileRefs(ID_ONE, files1);
+    rq1.addHFileRefs(ID_TWO, files1);
+    assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
+    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
+
+    rp.removePeer(ID_ONE);
+    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
+    assertNull(rqc.getReplicableHFiles(ID_ONE));
+    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
+
+    rp.removePeer(ID_TWO);
+    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
+    assertNull(rqc.getReplicableHFiles(ID_TWO));
+  }
+
+  @Test
   public void testReplicationPeers() throws Exception {
     rp.init();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 4587c61..3b7402a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -64,6 +64,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     utility = new HBaseTestingUtility();
     utility.startMiniZKCluster();
     conf = utility.getConfiguration();
+    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
     zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
     String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
     replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 13545b5..b36bb9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -52,15 +52,15 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
   private static final TableName t1_su = TableName.valueOf("t1_syncup");
   private static final TableName t2_su = TableName.valueOf("t2_syncup");
 
-  private static final byte[] famName = Bytes.toBytes("cf1");
+  protected static final byte[] famName = Bytes.toBytes("cf1");
   private static final byte[] qualName = Bytes.toBytes("q1");
 
-  private static final byte[] noRepfamName = Bytes.toBytes("norep");
+  protected static final byte[] noRepfamName = Bytes.toBytes("norep");
 
   private HTableDescriptor t1_syncupSource, t1_syncupTarget;
   private HTableDescriptor t2_syncupSource, t2_syncupTarget;
 
-  private Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
+  protected Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
 
   @Before
   public void setUp() throws Exception {
@@ -179,7 +179,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
 
   }
 
-  private void setupReplication() throws Exception {
+  protected void setupReplication() throws Exception {
     ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
     ReplicationAdmin admin2 = new ReplicationAdmin(conf2);
 
@@ -418,7 +418,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
     }
   }
 
-  private void syncUp(HBaseTestingUtility ut) throws Exception {
+  protected void syncUp(HBaseTestingUtility ut) throws Exception {
     ReplicationSyncUp.setConfigure(ut.getConfiguration());
     String[] arguments = new String[] { null };
     new ReplicationSyncUp().run(arguments);

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
new file mode 100644
index 0000000..f54c632
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpToolWithBulkLoadedData.java
@@ -0,0 +1,235 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.commons.logging.Log;
+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.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.HFileTestUtil;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplicationSyncUpTool {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestReplicationSyncUpToolWithBulkLoadedData.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+    conf1.set(HConstants.REPLICATION_CLUSTER_ID, "12345");
+    conf1.set("hbase.replication.source.fs.conf.provider",
+      TestSourceFSConfigurationProvider.class.getCanonicalName());
+    String classes = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
+    if (!classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint")) {
+      classes = classes + ",org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint";
+      conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, classes);
+    }
+
+    TestReplicationBase.setUpBeforeClass();
+  }
+
+  @Override
+  public void testSyncUpTool() throws Exception {
+    /**
+     * Set up Replication: on Master and one Slave Table: t1_syncup and t2_syncup columnfamily:
+     * 'cf1' : replicated 'norep': not replicated
+     */
+    setupReplication();
+
+    /**
+     * Prepare 16 random hfile ranges required for creating hfiles
+     */
+    Iterator<String> randomHFileRangeListIterator = null;
+    Set<String> randomHFileRanges = new HashSet<String>(16);
+    for (int i = 0; i < 16; i++) {
+      randomHFileRanges.add(UUID.randomUUID().toString());
+    }
+    List<String> randomHFileRangeList = new ArrayList<>(randomHFileRanges);
+    Collections.sort(randomHFileRangeList);
+    randomHFileRangeListIterator = randomHFileRangeList.iterator();
+
+    /**
+     * at Master: t1_syncup: Load 100 rows into cf1, and 3 rows into norep t2_syncup: Load 200 rows
+     * into cf1, and 3 rows into norep verify correctly replicated to slave
+     */
+    loadAndReplicateHFiles(true, randomHFileRangeListIterator);
+
+    /**
+     * Verify hfile load works step 1: stop hbase on Slave step 2: at Master: t1_syncup: Load
+     * another 100 rows into cf1 and 3 rows into norep t2_syncup: Load another 200 rows into cf1 and
+     * 3 rows into norep step 3: stop hbase on master, restart hbase on Slave step 4: verify Slave
+     * still has the rows before load t1_syncup: 100 rows from cf1 t2_syncup: 200 rows from cf1 step
+     * 5: run syncup tool on Master step 6: verify that hfiles show up on Slave and 'norep' does not
+     * t1_syncup: 200 rows from cf1 t2_syncup: 400 rows from cf1 verify correctly replicated to
+     * Slave
+     */
+    mimicSyncUpAfterBulkLoad(randomHFileRangeListIterator);
+
+  }
+
+  private void mimicSyncUpAfterBulkLoad(Iterator<String> randomHFileRangeListIterator)
+      throws Exception {
+    LOG.debug("mimicSyncUpAfterBulkLoad");
+    utility2.shutdownMiniHBaseCluster();
+
+    loadAndReplicateHFiles(false, randomHFileRangeListIterator);
+
+    int rowCount_ht1Source = utility1.countRows(ht1Source);
+    assertEquals("t1_syncup has 206 rows on source, after bulk load of another 103 hfiles", 206,
+      rowCount_ht1Source);
+
+    int rowCount_ht2Source = utility1.countRows(ht2Source);
+    assertEquals("t2_syncup has 406 rows on source, after bulk load of another 203 hfiles", 406,
+      rowCount_ht2Source);
+
+    utility1.shutdownMiniHBaseCluster();
+    utility2.restartHBaseCluster(1);
+
+    Thread.sleep(SLEEP_TIME);
+
+    // Before sync up
+    int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
+    int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
+    assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
+    assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
+
+    // Run sync up tool
+    syncUp(utility1);
+
+    // After syun up
+    for (int i = 0; i < NB_RETRIES; i++) {
+      syncUp(utility1);
+      rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
+      rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
+      if (i == NB_RETRIES - 1) {
+        if (rowCount_ht1TargetAtPeer1 != 200 || rowCount_ht2TargetAtPeer1 != 400) {
+          // syncUP still failed. Let's look at the source in case anything wrong there
+          utility1.restartHBaseCluster(1);
+          rowCount_ht1Source = utility1.countRows(ht1Source);
+          LOG.debug("t1_syncup should have 206 rows at source, and it is " + rowCount_ht1Source);
+          rowCount_ht2Source = utility1.countRows(ht2Source);
+          LOG.debug("t2_syncup should have 406 rows at source, and it is " + rowCount_ht2Source);
+        }
+        assertEquals("@Peer1 t1_syncup should be sync up and have 200 rows", 200,
+          rowCount_ht1TargetAtPeer1);
+        assertEquals("@Peer1 t2_syncup should be sync up and have 400 rows", 400,
+          rowCount_ht2TargetAtPeer1);
+      }
+      if (rowCount_ht1TargetAtPeer1 == 200 && rowCount_ht2TargetAtPeer1 == 400) {
+        LOG.info("SyncUpAfterBulkLoad succeeded at retry = " + i);
+        break;
+      } else {
+        LOG.debug("SyncUpAfterBulkLoad failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
+            + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
+            + rowCount_ht2TargetAtPeer1);
+      }
+      Thread.sleep(SLEEP_TIME);
+    }
+  }
+
+  private void loadAndReplicateHFiles(boolean verifyReplicationOnSlave,
+      Iterator<String> randomHFileRangeListIterator) throws Exception {
+    LOG.debug("loadAndReplicateHFiles");
+
+    // Load 100 + 3 hfiles to t1_syncup.
+    byte[][][] hfileRanges =
+        new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
+            Bytes.toBytes(randomHFileRangeListIterator.next()) } };
+    loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht1Source, hfileRanges,
+      100);
+
+    hfileRanges =
+        new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
+            Bytes.toBytes(randomHFileRangeListIterator.next()) } };
+    loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht1Source,
+      hfileRanges, 3);
+
+    // Load 200 + 3 hfiles to t2_syncup.
+    hfileRanges =
+        new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
+            Bytes.toBytes(randomHFileRangeListIterator.next()) } };
+    loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht2Source, hfileRanges,
+      200);
+
+    hfileRanges =
+        new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
+            Bytes.toBytes(randomHFileRangeListIterator.next()) } };
+    loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht2Source,
+      hfileRanges, 3);
+
+    if (verifyReplicationOnSlave) {
+      // ensure replication completed
+      wait(ht1TargetAtPeer1, utility1.countRows(ht1Source) - 3,
+        "t1_syncup has 103 rows on source, and 100 on slave1");
+
+      wait(ht2TargetAtPeer1, utility1.countRows(ht2Source) - 3,
+        "t2_syncup has 203 rows on source, and 200 on slave1");
+    }
+  }
+
+  private void loadAndValidateHFileReplication(String testName, byte[] row, byte[] fam,
+      Table source, byte[][][] hfileRanges, int numOfRows) throws Exception {
+    Path dir = utility1.getDataTestDirOnTestFS(testName);
+    FileSystem fs = utility1.getTestFileSystem();
+    dir = dir.makeQualified(fs);
+    Path familyDir = new Path(dir, Bytes.toString(fam));
+
+    int hfileIdx = 0;
+    for (byte[][] range : hfileRanges) {
+      byte[] from = range[0];
+      byte[] to = range[1];
+      HFileTestUtil.createHFile(utility1.getConfiguration(), fs, new Path(familyDir, "hfile_"
+          + hfileIdx++), fam, row, from, to, numOfRows);
+    }
+
+    final TableName tableName = source.getName();
+    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(utility1.getConfiguration());
+    String[] args = { dir.toString(), tableName.toString() };
+    loader.run(args);
+  }
+
+  private void wait(Table target, int expectedCount, String msg) throws IOException,
+      InterruptedException {
+    for (int i = 0; i < NB_RETRIES; i++) {
+      int rowCount_ht2TargetAtPeer1 = utility2.countRows(target);
+      if (i == NB_RETRIES - 1) {
+        assertEquals(msg, expectedCount, rowCount_ht2TargetAtPeer1);
+      }
+      if (expectedCount == rowCount_ht2TargetAtPeer1) {
+        break;
+      }
+      Thread.sleep(SLEEP_TIME);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index b87e7ef..f08d2bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -21,32 +21,52 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.security.SecureRandom;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionLocator;
 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.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -58,21 +78,18 @@ public class TestReplicationSink {
   private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
   private static final int BATCH_SIZE = 10;
 
-  private final static HBaseTestingUtility TEST_UTIL =
-      new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private static ReplicationSink SINK;
+  protected static ReplicationSink SINK;
 
-  private static final TableName TABLE_NAME1 =
-      TableName.valueOf("table1");
-  private static final TableName TABLE_NAME2 =
-      TableName.valueOf("table2");
+  protected static final TableName TABLE_NAME1 = TableName.valueOf("table1");
+  protected static final TableName TABLE_NAME2 = TableName.valueOf("table2");
 
-  private static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
-  private static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
+  protected static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
+  protected static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
 
-  private static Table table1;
-  private static Stoppable STOPPABLE = new Stoppable() {
+  protected static Table table1;
+  protected static Stoppable STOPPABLE = new Stoppable() {
     final AtomicBoolean stop = new AtomicBoolean(false);
 
     @Override
@@ -85,10 +102,13 @@ public class TestReplicationSink {
       LOG.info("STOPPING BECAUSE: " + why);
       this.stop.set(true);
     }
-    
+
   };
 
-  private static Table table2;
+  protected static Table table2;
+  protected static String baseNamespaceDir;
+  protected static String hfileArchiveDir;
+  protected static String replicationClusterId;
 
    /**
    * @throws java.lang.Exception
@@ -98,11 +118,18 @@ public class TestReplicationSink {
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
     TEST_UTIL.getConfiguration().setBoolean(HConstants.REPLICATION_ENABLE_KEY,
         HConstants.REPLICATION_ENABLE_DEFAULT);
+    TEST_UTIL.getConfiguration().set("hbase.replication.source.fs.conf.provider",
+      TestSourceFSConfigurationProvider.class.getCanonicalName());
+
     TEST_UTIL.startMiniCluster(3);
     SINK =
       new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
     table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1);
     table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2);
+    Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
+    baseNamespaceDir = new Path(rootDir, new Path(HConstants.BASE_NAMESPACE_DIR)).toString();
+    hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY)).toString();
+    replicationClusterId = "12345";
   }
 
   /**
@@ -134,7 +161,8 @@ public class TestReplicationSink {
     for(int i = 0; i < BATCH_SIZE; i++) {
       entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
     Scan scan = new Scan();
     ResultScanner scanRes = table1.getScanner(scan);
     assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length);
@@ -151,7 +179,8 @@ public class TestReplicationSink {
     for(int i = 0; i < BATCH_SIZE/2; i++) {
       entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells), replicationClusterId,
+      baseNamespaceDir, hfileArchiveDir);
 
     entries = new ArrayList<WALEntry>(BATCH_SIZE);
     cells = new ArrayList<Cell>();
@@ -160,7 +189,8 @@ public class TestReplicationSink {
           i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn, cells));
     }
 
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
     Scan scan = new Scan();
     ResultScanner scanRes = table1.getScanner(scan);
     assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length);
@@ -179,7 +209,8 @@ public class TestReplicationSink {
               i, KeyValue.Type.Put, cells));
     }
 
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
     Scan scan = new Scan();
     ResultScanner scanRes = table2.getScanner(scan);
     for(Result res : scanRes) {
@@ -198,14 +229,16 @@ public class TestReplicationSink {
     for(int i = 0; i < 3; i++) {
       entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
     entries = new ArrayList<WALEntry>(3);
     cells = new ArrayList<Cell>();
     entries.add(createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn, cells));
     entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
     entries.add(createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn, cells));
 
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
 
     Scan scan = new Scan();
     ResultScanner scanRes = table1.getScanner(scan);
@@ -228,12 +261,96 @@ public class TestReplicationSink {
     for(int i = 3; i < 5; i++) {
       entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
-    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
     Get get = new Get(Bytes.toBytes(1));
     Result res = table1.get(get);
     assertEquals(0, res.size());
   }
 
+  /**
+   * Test replicateEntries with a bulk load entry for 25 HFiles
+   */
+  @Test
+  public void testReplicateEntriesForHFiles() throws Exception {
+    Path dir = TEST_UTIL.getDataTestDirOnTestFS("testReplicateEntries");
+    Path familyDir = new Path(dir, Bytes.toString(FAM_NAME1));
+    int numRows = 10;
+
+    List<Path> p = new ArrayList<>(1);
+
+    // 1. Generate 25 hfile ranges
+    Random rng = new SecureRandom();
+    Set<Integer> numbers = new HashSet<>();
+    while (numbers.size() < 50) {
+      numbers.add(rng.nextInt(1000));
+    }
+    List<Integer> numberList = new ArrayList<>(numbers);
+    Collections.sort(numberList);
+
+    // 2. Create 25 hfiles
+    Configuration conf = TEST_UTIL.getConfiguration();
+    FileSystem fs = dir.getFileSystem(conf);
+    Iterator<Integer> numbersItr = numberList.iterator();
+    for (int i = 0; i < 25; i++) {
+      Path hfilePath = new Path(familyDir, "hfile_" + i);
+      HFileTestUtil.createHFile(conf, fs, hfilePath, FAM_NAME1, FAM_NAME1,
+        Bytes.toBytes(numbersItr.next()), Bytes.toBytes(numbersItr.next()), numRows);
+      p.add(hfilePath);
+    }
+
+    // 3. Create a BulkLoadDescriptor and a WALEdit
+    Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
+    storeFiles.put(FAM_NAME1, p);
+    WALEdit edit = null;
+    WALProtos.BulkLoadDescriptor loadDescriptor = null;
+
+    try (Connection c = ConnectionFactory.createConnection(conf);
+        RegionLocator l = c.getRegionLocator(TABLE_NAME1)) {
+      HRegionInfo regionInfo = l.getAllRegionLocations().get(0).getRegionInfo();
+      loadDescriptor =
+          ProtobufUtil.toBulkLoadDescriptor(TABLE_NAME1,
+            ByteStringer.wrap(regionInfo.getEncodedNameAsBytes()), storeFiles, 1);
+      edit = WALEdit.createBulkLoadEvent(regionInfo, loadDescriptor);
+    }
+    List<WALEntry> entries = new ArrayList<WALEntry>(1);
+
+    // 4. Create a WALEntryBuilder
+    WALEntry.Builder builder = createWALEntryBuilder(TABLE_NAME1);
+
+    // 5. Copy the hfile to the path as it is in reality
+    for (int i = 0; i < 25; i++) {
+      String pathToHfileFromNS =
+          new StringBuilder(100).append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR)
+              .append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR)
+              .append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray()))
+              .append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR)
+              .append("hfile_" + i).toString();
+      String dst = baseNamespaceDir + Path.SEPARATOR + pathToHfileFromNS;
+
+      FileUtil.copy(fs, p.get(0), fs, new Path(dst), false, conf);
+    }
+
+    entries.add(builder.build());
+    ResultScanner scanRes = null;
+    try {
+      Scan scan = new Scan();
+      scanRes = table1.getScanner(scan);
+      // 6. Assert no existing data in table
+      assertEquals(0, scanRes.next(numRows).length);
+      // 7. Replicate the bulk loaded entry
+      SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
+        replicationClusterId, baseNamespaceDir, hfileArchiveDir);
+      scanRes = table1.getScanner(scan);
+      // 8. Assert data is replicated
+      assertEquals(numRows, scanRes.next(numRows).length);
+    } finally {
+      if (scanRes != null) {
+        scanRes.close();
+      }
+    }
+  }
+
   private WALEntry createEntry(TableName table, int row,  KeyValue.Type type, List<Cell> cells) {
     byte[] fam = table.equals(TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2;
     byte[] rowBytes = Bytes.toBytes(row);
@@ -256,6 +373,13 @@ public class TestReplicationSink {
         kv = new KeyValue(rowBytes, fam, null,
             now, KeyValue.Type.DeleteFamily);
     }
+    WALEntry.Builder builder = createWALEntryBuilder(table);
+    cells.add(kv);
+
+    return builder.build();
+  }
+
+  private WALEntry.Builder createWALEntryBuilder(TableName table) {
     WALEntry.Builder builder = WALEntry.newBuilder();
     builder.setAssociatedCellCount(1);
     WALKey.Builder keyBuilder = WALKey.newBuilder();
@@ -264,13 +388,10 @@ public class TestReplicationSink {
     uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
     keyBuilder.setClusterId(uuidBuilder.build());
     keyBuilder.setTableName(ByteStringer.wrap(table.getName()));
-    keyBuilder.setWriteTime(now);
+    keyBuilder.setWriteTime(System.currentTimeMillis());
     keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY));
     keyBuilder.setLogSequenceNumber(-1);
     builder.setKey(keyBuilder.build());
-    cells.add(kv);
-
-    return builder.build();
+    return builder;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index d50522c..a208120 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -19,13 +19,17 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.net.URLEncoder;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -51,6 +55,8 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -64,6 +70,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -108,6 +115,8 @@ public class TestReplicationSourceManager {
 
   private static final byte[] f1 = Bytes.toBytes("f1");
 
+  private static final byte[] f2 = Bytes.toBytes("f2");
+
   private static final TableName test =
       TableName.valueOf("test");
 
@@ -161,10 +170,10 @@ public class TestReplicationSourceManager {
     manager.addSource(slaveId);
 
     htd = new HTableDescriptor(test);
-    HColumnDescriptor col = new HColumnDescriptor("f1");
+    HColumnDescriptor col = new HColumnDescriptor(f1);
     col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
     htd.addFamily(col);
-    col = new HColumnDescriptor("f2");
+    col = new HColumnDescriptor(f2);
     col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
     htd.addFamily(col);
 
@@ -416,6 +425,63 @@ public class TestReplicationSourceManager {
     s0.abort("", null);
   }
 
+  @Test
+  public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
+    // 1. Create wal key
+    WALKey logKey = new WALKey();
+    // 2. Get the bulk load wal edit event
+    WALEdit logEdit = getBulkLoadWALEdit();
+
+    // 3. Get the scopes for the key
+    Replication.scopeWALEdits(htd, logKey, logEdit, conf, manager);
+
+    // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
+    assertNull("No bulk load entries scope should be added if bulk load replication is diabled.",
+      logKey.getScopes());
+  }
+
+  @Test
+  public void testBulkLoadWALEdits() throws Exception {
+    // 1. Create wal key
+    WALKey logKey = new WALKey();
+    // 2. Get the bulk load wal edit event
+    WALEdit logEdit = getBulkLoadWALEdit();
+    // 3. Enable bulk load hfile replication
+    Configuration bulkLoadConf = HBaseConfiguration.create(conf);
+    bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+
+    // 4. Get the scopes for the key
+    Replication.scopeWALEdits(htd, logKey, logEdit, bulkLoadConf, manager);
+
+    NavigableMap<byte[], Integer> scopes = logKey.getScopes();
+    // Assert family with replication scope global is present in the key scopes
+    assertTrue("This family scope is set to global, should be part of replication key scopes.",
+      scopes.containsKey(f1));
+    // Assert family with replication scope local is not present in the key scopes
+    assertFalse("This family scope is set to local, should not be part of replication key scopes",
+      scopes.containsKey(f2));
+  }
+
+  private WALEdit getBulkLoadWALEdit() {
+    // 1. Create store files for the families
+    Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
+    List<Path> p = new ArrayList<>(1);
+    p.add(new Path(Bytes.toString(f1)));
+    storeFiles.put(f1, p);
+
+    p = new ArrayList<>(1);
+    p.add(new Path(Bytes.toString(f2)));
+    storeFiles.put(f2, p);
+
+    // 2. Create bulk load descriptor
+    BulkLoadDescriptor desc = ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
+      ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, 1);
+
+    // 3. create bulk load wal edit event
+    WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
+    return logEdit;
+  }
+
   static class DummyNodeFailoverWorker extends Thread {
     private SortedMap<String, SortedSet<String>> logZnodesMap;
     Server server;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26ac60b0/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSourceFSConfigurationProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSourceFSConfigurationProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSourceFSConfigurationProvider.java
new file mode 100644
index 0000000..a14c02b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSourceFSConfigurationProvider.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class TestSourceFSConfigurationProvider implements SourceFSConfigurationProvider {
+  @Override
+  public Configuration getConf(Configuration sinkConf, String replicationClusterId)
+      throws IOException {
+    return sinkConf;
+  }
+}


[14/26] hbase git commit: HBASE-14946 Don't allow multi's to over run the max result size.

Posted by sy...@apache.org.
HBASE-14946 Don't allow multi's to over run the max result size.

Summary:
* Add VersionInfoUtil to determine if a client has a specified version or better
* Add an exception type to say that the response should be chunked
* Add on client knowledge of retry exceptions
* Add on metrics for how often this happens

Test Plan: Added a unit test

Differential Revision: https://reviews.facebook.net/D51771


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

Branch: refs/heads/hbase-12439
Commit: 48e217a7db8c23501ea4934d28e57684b82d71fb
Parents: c15e0af
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Dec 7 18:33:35 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Dec 10 18:10:32 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/MultiActionResultTooLarge.java |  31 ++++
 .../hadoop/hbase/RetryImmediatelyException.java |  27 ++++
 .../hadoop/hbase/client/AsyncProcess.java       |  86 ++++++++---
 .../hbase/client/ConnectionImplementation.java  |   7 +-
 .../org/apache/hadoop/hbase/client/Result.java  |   3 +
 .../hbase/ipc/MetricsHBaseServerSource.java     |   8 +-
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java |   9 ++
 .../hadoop/hbase/client/VersionInfoUtil.java    |  63 ++++++++
 .../hadoop/hbase/ipc/MetricsHBaseServer.java    |   3 +
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |  23 ++-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  27 +++-
 .../master/procedure/ProcedurePrepareLatch.java |  23 +--
 .../hbase/regionserver/RSRpcServices.java       | 154 ++++++++++++-------
 .../hbase/client/TestMultiRespectsLimits.java   | 102 ++++++++++++
 14 files changed, 462 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
new file mode 100644
index 0000000..d06eea1
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MultiActionResultTooLarge.java
@@ -0,0 +1,31 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hbase;
+
+/**
+ * Exception thrown when the result needs to be chunked on the server side.
+ * It signals that retries should happen right away and not count against the number of
+ * retries because some of the multi was a success.
+ */
+public class MultiActionResultTooLarge extends RetryImmediatelyException {
+
+  public MultiActionResultTooLarge(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
new file mode 100644
index 0000000..1b39904
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RetryImmediatelyException.java
@@ -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.
+ */
+
+package org.apache.hadoop.hbase;
+
+import java.io.IOException;
+
+public class RetryImmediatelyException extends IOException {
+  public RetryImmediatelyException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index f1fa3eb..5102ec5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.RetryImmediatelyException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -126,19 +127,35 @@ class AsyncProcess {
     public void waitUntilDone() throws InterruptedIOException;
   }
 
-  /** Return value from a submit that didn't contain any requests. */
+  /**
+   * Return value from a submit that didn't contain any requests.
+   */
   private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
     final Object[] result = new Object[0];
+
     @Override
-    public boolean hasError() { return false; }
+    public boolean hasError() {
+      return false;
+    }
+
     @Override
-    public RetriesExhaustedWithDetailsException getErrors() { return null; }
+    public RetriesExhaustedWithDetailsException getErrors() {
+      return null;
+    }
+
     @Override
-    public List<? extends Row> getFailedOperations() { return null; }
+    public List<? extends Row> getFailedOperations() {
+      return null;
+    }
+
     @Override
-    public Object[] getResults() { return result; }
+    public Object[] getResults() {
+      return result;
+    }
+
     @Override
-    public void waitUntilDone() throws InterruptedIOException {}
+    public void waitUntilDone() throws InterruptedIOException {
+    }
   };
 
   /** Sync point for calls to multiple replicas for the same user request (Get).
@@ -308,8 +325,12 @@ class AsyncProcess {
    *         RuntimeException
    */
   private ExecutorService getPool(ExecutorService pool) {
-    if (pool != null) return pool;
-    if (this.pool != null) return this.pool;
+    if (pool != null) {
+      return pool;
+    }
+    if (this.pool != null) {
+      return this.pool;
+    }
     throw new RuntimeException("Neither AsyncProcess nor request have ExecutorService");
   }
 
@@ -367,7 +388,9 @@ class AsyncProcess {
         Row r = it.next();
         HRegionLocation loc;
         try {
-          if (r == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
+          if (r == null) {
+            throw new IllegalArgumentException("#" + id + ", row cannot be null");
+          }
           // Make sure we get 0-s replica.
           RegionLocations locs = connection.locateRegion(
               tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID);
@@ -730,10 +753,10 @@ class AsyncProcess {
           // Normal case: we received an answer from the server, and it's not an exception.
           receiveMultiAction(multiAction, server, res, numAttempt);
         } catch (Throwable t) {
-              // Something really bad happened. We are on the send thread that will now die.
-              LOG.error("Internal AsyncProcess #" + id + " error for "
-                  + tableName + " processing for " + server, t);
-              throw new RuntimeException(t);
+          // Something really bad happened. We are on the send thread that will now die.
+          LOG.error("Internal AsyncProcess #" + id + " error for "
+              + tableName + " processing for " + server, t);
+          throw new RuntimeException(t);
         } finally {
           decTaskCounters(multiAction.getRegions(), server);
           if (callsInProgress != null && callable != null) {
@@ -752,19 +775,25 @@ class AsyncProcess {
 
     private final TableName tableName;
     private final AtomicLong actionsInProgress = new AtomicLong(-1);
-    /** The lock controls access to results. It is only held when populating results where
+    /**
+     * The lock controls access to results. It is only held when populating results where
      * there might be several callers (eventual consistency gets). For other requests,
-     * there's one unique call going on per result index. */
+     * there's one unique call going on per result index.
+     */
     private final Object replicaResultLock = new Object();
-    /** Result array.  Null if results are not needed. Otherwise, each index corresponds to
+    /**
+     * Result array.  Null if results are not needed. Otherwise, each index corresponds to
      * the action index in initial actions submitted. For most request types, has null-s for
      * requests that are not done, and result/exception for those that are done.
      * For eventual-consistency gets, initially the same applies; at some point, replica calls
      * might be started, and ReplicaResultState is put at the corresponding indices. The
      * returning calls check the type to detect when this is the case. After all calls are done,
-     * ReplicaResultState-s are replaced with results for the user. */
+     * ReplicaResultState-s are replaced with results for the user.
+     */
     private final Object[] results;
-    /** Indices of replica gets in results. If null, all or no actions are replica-gets. */
+    /**
+     * Indices of replica gets in results. If null, all or no actions are replica-gets.
+     */
     private final int[] replicaGetIndices;
     private final boolean hasAnyReplicaGets;
     private final long nonceGroup;
@@ -779,7 +808,9 @@ class AsyncProcess {
       this.actionsInProgress.set(actions.size());
       if (results != null) {
         assert needResults;
-        if (results.length != actions.size()) throw new AssertionError("results.length");
+        if (results.length != actions.size()) {
+          throw new AssertionError("results.length");
+        }
         this.results = results;
         for (int i = 0; i != this.results.length; ++i) {
           results[i] = null;
@@ -1178,9 +1209,13 @@ class AsyncProcess {
       // We have two contradicting needs here:
       //  1) We want to get the new location after having slept, as it may change.
       //  2) We want to take into account the location when calculating the sleep time.
+      //  3) If all this is just because the response needed to be chunked try again FAST.
       // It should be possible to have some heuristics to take the right decision. Short term,
       //  we go for one.
-      long backOffTime = errorsByServer.calculateBackoffTime(oldServer, pause);
+      boolean retryImmediately = throwable instanceof RetryImmediatelyException;
+      int nextAttemptNumber = retryImmediately ? numAttempt : numAttempt + 1;
+      long backOffTime = retryImmediately ? 0 :
+          errorsByServer.calculateBackoffTime(oldServer, pause);
       if (numAttempt > startLogErrorsCnt) {
         // We use this value to have some logs when we have multiple failures, but not too many
         //  logs, as errors are to be expected when a region moves, splits and so on
@@ -1189,14 +1224,16 @@ class AsyncProcess {
       }
 
       try {
-        Thread.sleep(backOffTime);
+        if (backOffTime > 0) {
+          Thread.sleep(backOffTime);
+        }
       } catch (InterruptedException e) {
         LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e);
         Thread.currentThread().interrupt();
         return;
       }
 
-      groupAndSendMultiAction(toReplay, numAttempt + 1);
+      groupAndSendMultiAction(toReplay, nextAttemptNumber);
     }
 
     private void logNoResubmit(ServerName oldServer, int numAttempt,
@@ -1256,6 +1293,7 @@ class AsyncProcess {
           // Failure: retry if it's make sense else update the errors lists
           if (result == null || result instanceof Throwable) {
             Row row = sentAction.getAction();
+            throwable = ConnectionImplementation.findException(result);
             // Register corresponding failures once per server/once per region.
             if (!regionFailureRegistered) {
               regionFailureRegistered = true;
@@ -1405,7 +1443,9 @@ class AsyncProcess {
       // will either see state with callCount 0 after locking it; or will not see state at all
       // we will replace it with the result.
       synchronized (state) {
-        if (state.callCount == 0) return; // someone already set the result
+        if (state.callCount == 0) {
+          return; // someone already set the result
+        }
         state.callCount = 0;
       }
       synchronized (replicaResultLock) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 62a7998..0ef2a17 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -35,8 +35,10 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.MultiActionResultTooLarge;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.RegionTooBusyException;
+import org.apache.hadoop.hbase.RetryImmediatelyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -298,7 +300,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     Throwable cur = (Throwable) exception;
     while (cur != null) {
       if (cur instanceof RegionMovedException || cur instanceof RegionOpeningException
-          || cur instanceof RegionTooBusyException || cur instanceof ThrottlingException) {
+          || cur instanceof RegionTooBusyException || cur instanceof ThrottlingException
+          || cur instanceof RetryImmediatelyException) {
         return cur;
       }
       if (cur instanceof RemoteException) {
@@ -1929,7 +1932,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     Throwable cause = findException(exception);
     if (cause != null) {
       if (cause instanceof RegionTooBusyException || cause instanceof RegionOpeningException
-          || cause instanceof ThrottlingException) {
+          || cause instanceof ThrottlingException || cause instanceof MultiActionResultTooLarge) {
         // We know that the region is still on this region server
         return;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index 702983b..d2a49c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -839,6 +839,9 @@ public class Result implements CellScannable, CellScanner {
    */
   public static long getTotalSizeOfCells(Result result) {
     long size = 0;
+    if (result.isEmpty()) {
+      return size;
+    }
     for (Cell c : result.rawCells()) {
       size += CellUtil.estimatedHeapSizeOf(c);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
index 5cf71f3..061a672 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
@@ -74,6 +74,9 @@ public interface MetricsHBaseServerSource extends BaseSource {
   String EXCEPTIONS_SANITY_NAME="exceptions.FailedSanityCheckException";
   String EXCEPTIONS_MOVED_NAME="exceptions.RegionMovedException";
   String EXCEPTIONS_NSRE_NAME="exceptions.NotServingRegionException";
+  String EXCEPTIONS_MULTI_TOO_LARGE_NAME = "exceptions.multiResponseTooLarge";
+  String EXCEPTIONS_MULTI_TOO_LARGE_DESC = "A response to a multi request was too large and the " +
+      "rest of the requests will have to be retried.";
 
   void authorizationSuccess();
 
@@ -96,6 +99,7 @@ public interface MetricsHBaseServerSource extends BaseSource {
   void notServingRegionException();
   void unknownScannerException();
   void tooBusyException();
+  void multiActionTooLargeException();
 
   void sentBytes(long count);
 
@@ -110,4 +114,6 @@ public interface MetricsHBaseServerSource extends BaseSource {
   void processedCall(int processingTime);
 
   void queuedAndProcessedCall(int totalTime);
-  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index 8984394..487f9f5 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.metrics2.lib.MutableHistogram;
 public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
     implements MetricsHBaseServerSource {
 
+
   private final MetricsHBaseServerWrapper wrapper;
   private final MutableCounterLong authorizationSuccesses;
   private final MutableCounterLong authorizationFailures;
@@ -47,6 +48,7 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
   private final MutableCounterLong exceptionsSanity;
   private final MutableCounterLong exceptionsNSRE;
   private final MutableCounterLong exceptionsMoved;
+  private final MutableCounterLong exceptionsMultiTooLarge;
 
 
   private MutableHistogram queueCallTime;
@@ -81,6 +83,8 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
         .newCounter(EXCEPTIONS_MOVED_NAME, EXCEPTIONS_TYPE_DESC, 0L);
     this.exceptionsNSRE = this.getMetricsRegistry()
         .newCounter(EXCEPTIONS_NSRE_NAME, EXCEPTIONS_TYPE_DESC, 0L);
+    this.exceptionsMultiTooLarge = this.getMetricsRegistry()
+        .newCounter(EXCEPTIONS_MULTI_TOO_LARGE_NAME, EXCEPTIONS_MULTI_TOO_LARGE_DESC, 0L);
 
     this.authenticationSuccesses = this.getMetricsRegistry().newCounter(
         AUTHENTICATION_SUCCESSES_NAME, AUTHENTICATION_SUCCESSES_DESC, 0L);
@@ -160,6 +164,11 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
   }
 
   @Override
+  public void multiActionTooLargeException() {
+    exceptionsMultiTooLarge.incr();
+  }
+
+  @Override
   public void authenticationSuccess() {
     authenticationSuccesses.incr();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
new file mode 100644
index 0000000..c405518
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
@@ -0,0 +1,63 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.RpcCallContext;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+
+
+/**
+ * Class to help with parsing the version info.
+ */
+@InterfaceAudience.Private
+public final class VersionInfoUtil {
+
+  private VersionInfoUtil() {
+    /* UTIL CLASS ONLY */
+  }
+
+  public static boolean currentClientHasMinimumVersion(int major, int minor) {
+    RpcCallContext call = RpcServer.getCurrentCall();
+    HBaseProtos.VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
+    return hasMinimumVersion(versionInfo, major, minor);
+  }
+
+  public static boolean hasMinimumVersion(HBaseProtos.VersionInfo versionInfo,
+                                          int major,
+                                          int minor) {
+    if (versionInfo != null) {
+      try {
+        String[] components = versionInfo.getVersion().split("\\.");
+
+        int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
+        if (clientMajor != major) {
+          return clientMajor > major;
+        }
+
+        int clientMinor = components.length > 1 ? Integer.parseInt(components[1]) : 0;
+        return clientMinor >= minor;
+      } catch (NumberFormatException e) {
+        return false;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
index d276503..05bebb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.MultiActionResultTooLarge;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.UnknownScannerException;
@@ -105,6 +106,8 @@ public class MetricsHBaseServer {
         source.notServingRegionException();
       } else if (throwable instanceof FailedSanityCheckException) {
         source.failedSanityException();
+      } else if (throwable instanceof MultiActionResultTooLarge) {
+        source.multiActionTooLargeException();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index 60e5f5d..d14e9b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -19,10 +19,11 @@ package org.apache.hadoop.hbase.ipc;
 
 import java.net.InetAddress;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
 import org.apache.hadoop.hbase.security.User;
 
-
+@InterfaceAudience.Private
 public interface RpcCallContext extends Delayable {
   /**
    * Check if the caller who made this IPC call has disconnected.
@@ -40,7 +41,7 @@ public interface RpcCallContext extends Delayable {
    * support cellblocks while fielding requests from clients that do not.
    * @return True if the client supports cellblocks, else return all content in pb
    */
-  boolean isClientCellBlockSupport();
+  boolean isClientCellBlockSupported();
 
   /**
    * Returns the user credentials associated with the current RPC request or
@@ -71,4 +72,22 @@ public interface RpcCallContext extends Delayable {
    * @param callback
    */
   void setCallBack(RpcCallback callback);
+
+  boolean isRetryImmediatelySupported();
+
+  /**
+   * The size of response cells that have been accumulated so far.
+   * This along with the corresponding increment call is used to ensure that multi's or
+   * scans dont get too excessively large
+   */
+  long getResponseCellSize();
+
+  /**
+   * Add on the given amount to the retained cell size.
+   *
+   * This is not thread safe and not synchronized at all. If this is used by more than one thread
+   * then everything will break. Since this is called for every row synchronization would be too
+   * onerous.
+   */
+  void incrementResponseCellSize(long cellSize);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 0db7383..2bef247 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Operation;
+import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
@@ -317,6 +318,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     private InetAddress remoteAddress;
     private RpcCallback callback;
 
+    private long responseCellSize = 0;
+    private boolean retryImmediatelySupported;
+
     Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
          Message param, CellScanner cellScanner, Connection connection, Responder responder,
          long size, TraceInfo tinfo, final InetAddress remoteAddress) {
@@ -336,6 +340,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       this.tinfo = tinfo;
       this.user = connection.user;
       this.remoteAddress = remoteAddress;
+      this.retryImmediatelySupported = connection.retryImmediatelySupported;
     }
 
     /**
@@ -521,7 +526,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     }
 
     @Override
-    public boolean isClientCellBlockSupport() {
+    public boolean isClientCellBlockSupported() {
       return this.connection != null && this.connection.codec != null;
     }
 
@@ -538,6 +543,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       return this.size;
     }
 
+    public long getResponseCellSize() {
+      return responseCellSize;
+    }
+
+    public void incrementResponseCellSize(long cellSize) {
+      responseCellSize += cellSize;
+    }
+
     /**
      * If we have a response, and delay is not set, then respond
      * immediately.  Otherwise, do not respond to client.  This is
@@ -578,6 +591,11 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     public void setCallBack(RpcCallback callback) {
       this.callback = callback;
     }
+
+    @Override
+    public boolean isRetryImmediatelySupported() {
+      return retryImmediatelySupported;
+    }
   }
 
   /** Listens on the socket. Creates jobs for the handler threads*/
@@ -1264,6 +1282,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     // was authentication allowed with a fallback to simple auth
     private boolean authenticatedWithFallback;
 
+    private boolean retryImmediatelySupported = false;
+
     public UserGroupInformation attemptingUser = null; // user name before auth
     protected User user = null;
     protected UserGroupInformation ugi = null;
@@ -1720,6 +1740,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
       }
       if (connectionHeader.hasVersionInfo()) {
+        // see if this connection will support RetryImmediatelyException
+        retryImmediatelySupported = VersionInfoUtil.hasMinimumVersion(getVersionInfo(), 1, 2);
+
         AUDITLOG.info("Connection from " + this.hostAddress + " port: " + this.remotePort
             + " with version info: "
             + TextFormat.shortDebugString(connectionHeader.getVersionInfo()));
@@ -1727,6 +1750,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         AUDITLOG.info("Connection from " + this.hostAddress + " port: " + this.remotePort
             + " with unknown version info");
       }
+
+
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
index 052386a..b13e44d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -24,10 +24,8 @@ import java.util.concurrent.CountDownLatch;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.ipc.RpcServer;
-import org.apache.hadoop.hbase.ipc.RpcCallContext;
+import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
 
 /**
  * Latch used by the Master to have the prepare() sync behaviour for old
@@ -44,24 +42,7 @@ public abstract class ProcedurePrepareLatch {
   }
 
   public static boolean hasProcedureSupport() {
-    return currentClientHasMinimumVersion(1, 1);
-  }
-
-  private static boolean currentClientHasMinimumVersion(int major, int minor) {
-    RpcCallContext call = RpcServer.getCurrentCall();
-    VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
-    if (versionInfo != null) {
-      String[] components = versionInfo.getVersion().split("\\.");
-
-      int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
-      if (clientMajor != major) {
-        return clientMajor > major;
-      }
-
-      int clientMinor = components.length > 1 ? Integer.parseInt(components[1]) : 0;
-      return clientMinor >= minor;
-    }
-    return false;
+    return VersionInfoUtil.currentClientHasMinimumVersion(1, 1);
   }
 
   protected abstract void countDown(final Procedure proc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 0c9b0e6..bba38f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MultiActionResultTooLarge;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -437,11 +438,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private boolean isClientCellBlockSupport() {
     RpcCallContext context = RpcServer.getCurrentCall();
-    return context != null && context.isClientCellBlockSupport();
+    return context != null && context.isClientCellBlockSupported();
   }
 
   private boolean isClientCellBlockSupport(RpcCallContext context) {
-    return context != null && context.isClientCellBlockSupport();
+    return context != null && context.isClientCellBlockSupported();
   }
 
   private void addResult(final MutateResponse.Builder builder, final Result result,
@@ -500,13 +501,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         rm = new RowMutations(action.getMutation().getRow().toByteArray());
       }
       switch (type) {
-      case PUT:
-        rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
-        break;
-      case DELETE:
-        rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
-        break;
-      default:
+        case PUT:
+          rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
+          break;
+        case DELETE:
+          rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
+          break;
+        default:
           throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
       }
     }
@@ -543,14 +544,14 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         rm = new RowMutations(action.getMutation().getRow().toByteArray());
       }
       switch (type) {
-      case PUT:
-        rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
-        break;
-      case DELETE:
-        rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
-        break;
-      default:
-        throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
+        case PUT:
+          rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
+          break;
+        case DELETE:
+          rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
+          break;
+        default:
+          throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name());
       }
     }
     return region.checkAndRowMutate(row, family, qualifier, compareOp, comparator, rm, Boolean.TRUE);
@@ -655,10 +656,42 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     // ResultOrException instance that matches each Put or Delete is then added down in the
     // doBatchOp call.  We should be staying aligned though the Put and Delete are deferred/batched
     List<ClientProtos.Action> mutations = null;
+    long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
+    IOException sizeIOE = null;
     for (ClientProtos.Action action : actions.getActionList()) {
       ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null;
       try {
         Result r = null;
+
+        if (context != null
+            && context.isRetryImmediatelySupported()
+            && context.getResponseCellSize() > maxQuotaResultSize) {
+
+          // We're storing the exception since the exception and reason string won't
+          // change after the response size limit is reached.
+          if (sizeIOE == null ) {
+            // We don't need the stack un-winding do don't throw the exception.
+            // Throwing will kill the JVM's JIT.
+            //
+            // Instead just create the exception and then store it.
+            sizeIOE = new MultiActionResultTooLarge("Max response size exceeded: "
+                    + context.getResponseCellSize());
+
+            // Only report the exception once since there's only one request that
+            // caused the exception. Otherwise this number will dominate the exceptions count.
+            rpcServer.getMetrics().exception(sizeIOE);
+          }
+
+          // Now that there's an exception is know to be created
+          // use it for the response.
+          //
+          // This will create a copy in the builder.
+          resultOrExceptionBuilder = ResultOrException.newBuilder().
+              setException(ResponseConverter.buildException(sizeIOE));
+          resultOrExceptionBuilder.setIndex(action.getIndex());
+          builder.addResultOrException(resultOrExceptionBuilder.build());
+          continue;
+        }
         if (action.hasGet()) {
           Get get = ProtobufUtil.toGet(action.getGet());
           if (context != null) {
@@ -690,22 +723,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             mutations.clear();
           }
           switch (type) {
-          case APPEND:
-            r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
-            break;
-          case INCREMENT:
-            r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
-            break;
-          case PUT:
-          case DELETE:
-            // Collect the individual mutations and apply in a batch
-            if (mutations == null) {
-              mutations = new ArrayList<ClientProtos.Action>(actions.getActionCount());
-            }
-            mutations.add(action);
-            break;
-          default:
-            throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
+            case APPEND:
+              r = append(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              break;
+            case INCREMENT:
+              r = increment(region, quota, action.getMutation(), cellScanner, nonceGroup);
+              break;
+            case PUT:
+            case DELETE:
+              // Collect the individual mutations and apply in a batch
+              if (mutations == null) {
+                mutations = new ArrayList<ClientProtos.Action>(actions.getActionCount());
+              }
+              mutations.add(action);
+              break;
+            default:
+              throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
           }
         } else {
           throw new HBaseIOException("Unexpected Action type");
@@ -715,11 +748,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           if (isClientCellBlockSupport(context)) {
             pbResult = ProtobufUtil.toResultNoData(r);
             //  Hard to guess the size here.  Just make a rough guess.
-            if (cellsToReturn == null) cellsToReturn = new ArrayList<CellScannable>();
+            if (cellsToReturn == null) {
+              cellsToReturn = new ArrayList<CellScannable>();
+            }
             cellsToReturn.add(r);
           } else {
             pbResult = ProtobufUtil.toResult(r);
           }
+          if (context != null) {
+            context.incrementResponseCellSize(Result.getTotalSizeOfCells(r));
+          }
           resultOrExceptionBuilder =
             ClientProtos.ResultOrException.newBuilder().setResult(pbResult);
         }
@@ -801,8 +839,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
           case SUCCESS:
             builder.addResultOrException(getResultOrException(
-              ClientProtos.Result.getDefaultInstance(), index,
-                ((HRegion)region).getRegionStats()));
+                ClientProtos.Result.getDefaultInstance(), index,
+                ((HRegion) region).getRegionStats()));
             break;
         }
       }
@@ -951,13 +989,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     ConnectionUtils.setServerSideHConnectionRetriesConfig(rs.conf, name, LOG);
     try {
       rpcServer = new RpcServer(rs, name, getServices(),
-        bindAddress, // use final bindAddress for this server.
-        rs.conf,
-        rpcSchedulerFactory.create(rs.conf, this, rs));
-    } catch(BindException be) {
+          bindAddress, // use final bindAddress for this server.
+          rs.conf,
+          rpcSchedulerFactory.create(rs.conf, this, rs));
+    } catch (BindException be) {
       String configName = (this instanceof MasterRpcServices) ? HConstants.MASTER_PORT :
-        HConstants.REGIONSERVER_PORT;
-        throw new IOException(be.getMessage() + ". To switch ports use the '" + configName +
+          HConstants.REGIONSERVER_PORT;
+      throw new IOException(be.getMessage() + ". To switch ports use the '" + configName +
           "' configuration property.", be.getCause() != null ? be.getCause() : be);
     }
 
@@ -2106,7 +2144,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     // It is also the conduit via which we pass back data.
     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
     CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
-    if (controller != null) controller.setCellScanner(null);
+    if (controller != null) {
+      controller.setCellScanner(null);
+    }
 
     long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
 
@@ -2180,7 +2220,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     if (cellsToReturn != null && !cellsToReturn.isEmpty() && controller != null) {
       controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn));
     }
-    if (processed != null) responseBuilder.setProcessed(processed);
+    if (processed != null) {
+      responseBuilder.setProcessed(processed);
+    }
     return responseBuilder.build();
   }
 
@@ -2197,10 +2239,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
     // It is also the conduit via which we pass back data.
     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
-    CellScanner cellScanner = controller != null? controller.cellScanner(): null;
+    CellScanner cellScanner = controller != null ? controller.cellScanner() : null;
     OperationQuota quota = null;
     // Clear scanner so we are not holding on to reference across call.
-    if (controller != null) controller.setCellScanner(null);
+    if (controller != null) {
+      controller.setCellScanner(null);
+    }
     try {
       checkOpen();
       requestCount.increment();
@@ -2448,8 +2492,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           // where processing of request takes > lease expiration time.
           lease = regionServer.leases.removeLease(scannerName);
           List<Result> results = new ArrayList<Result>();
-          long totalCellSize = 0;
-          long currentScanResultSize = 0;
 
           boolean done = false;
           // Call coprocessor. Get region info from scanner.
@@ -2459,8 +2501,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             if (!results.isEmpty()) {
               for (Result r : results) {
                 for (Cell cell : r.rawCells()) {
-                  totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
-                  currentScanResultSize += CellUtil.estimatedHeapSizeOf(cell);
+                  if (context != null) {
+                    context.incrementResponseCellSize(CellUtil.estimatedSerializedSizeOf(cell));
+                  }
                 }
               }
             }
@@ -2493,7 +2536,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 // If the coprocessor host is adding to the result list, we cannot guarantee the
                 // correct ordering of partial results and so we prevent partial results from being
                 // formed.
-                boolean serverGuaranteesOrderOfPartials = currentScanResultSize == 0;
+                boolean serverGuaranteesOrderOfPartials = results.isEmpty();
                 boolean allowPartialResults =
                     clientHandlesPartials && serverGuaranteesOrderOfPartials && !isSmallScan;
                 boolean moreRows = false;
@@ -2559,7 +2602,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
                   if (!values.isEmpty()) {
                     for (Cell cell : values) {
-                      totalCellSize += CellUtil.estimatedSerializedSizeOf(cell);
+                      if (context != null) {
+                        context.incrementResponseCellSize(CellUtil.estimatedSerializedSizeOf(cell));
+                      }
                     }
                     final boolean partial = scannerContext.partialResultFormed();
                     results.add(Result.create(values, null, stale, partial));
@@ -2614,9 +2659,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                 }
               }
               region.updateReadRequestsCount(i);
-              region.getMetrics().updateScanNext(totalCellSize);
+              long responseCellSize = context != null ? context.getResponseCellSize() : 0;
+              region.getMetrics().updateScanNext(responseCellSize);
               if (regionServer.metricsRegionServer != null) {
-                regionServer.metricsRegionServer.updateScannerNext(totalCellSize);
+                regionServer.metricsRegionServer.updateScannerNext(responseCellSize);
               }
             } finally {
               region.closeRegionOperation();

http://git-wip-us.apache.org/repos/asf/hbase/blob/48e217a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
new file mode 100644
index 0000000..47dd7be
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
@@ -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.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.CompatibilityFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static junit.framework.TestCase.assertEquals;
+
+/**
+ * This test sets the multi size WAAAAAY low and then checks to make sure that gets will still make
+ * progress.
+ */
+@Category({MediumTests.class, ClientTests.class})
+public class TestMultiRespectsLimits {
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final MetricsAssertHelper METRICS_ASSERT =
+      CompatibilityFactory.getInstance(MetricsAssertHelper.class);
+  private final static byte[] FAMILY = Bytes.toBytes("D");
+  public static final int MAX_SIZE = 500;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setLong(
+        HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY,
+        MAX_SIZE);
+
+    // Only start on regionserver so that all regions are on the same server.
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testMultiLimits() throws Exception {
+    final TableName name = TableName.valueOf("testMultiLimits");
+    Table t = TEST_UTIL.createTable(name, FAMILY);
+    TEST_UTIL.loadTable(t, FAMILY, false);
+
+    // Split the table to make sure that the chunking happens accross regions.
+    try (final Admin admin = TEST_UTIL.getHBaseAdmin()) {
+      admin.split(name);
+      TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return admin.getTableRegions(name).size() > 1;
+        }
+      });
+    }
+    List<Get> gets = new ArrayList<>(MAX_SIZE);
+
+    for (int i = 0; i < MAX_SIZE; i++) {
+      gets.add(new Get(HBaseTestingUtility.ROWS[i]));
+    }
+    Result[] results = t.get(gets);
+    assertEquals(MAX_SIZE, results.length);
+    RpcServerInterface rpcServer = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer();
+    BaseSource s = rpcServer.getMetrics().getMetricsSource();
+
+    // Cells from TEST_UTIL.loadTable have a length of 27.
+    // Multiplying by less than that gives an easy lower bound on size.
+    // However in reality each kv is being reported as much higher than that.
+    METRICS_ASSERT.assertCounterGt("exceptions", (MAX_SIZE * 25) / MAX_SIZE, s);
+    METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge",
+        (MAX_SIZE * 25) / MAX_SIZE, s);
+  }
+}


[19/26] hbase git commit: HBASE-14701 Fix flakey Failed tests: TestMobFlushSnapshotFromClient>TestFlushSnapshotFromClient.testSkipFlushTableSnapshot:199 null (Jingcheng Du)

Posted by sy...@apache.org.
HBASE-14701 Fix flakey Failed tests: TestMobFlushSnapshotFromClient>TestFlushSnapshotFromClient.testSkipFlushTableSnapshot:199 null (Jingcheng Du)


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

Branch: refs/heads/hbase-12439
Commit: f34d3e1d2634566959778b4ecf1fcb5cb20d8721
Parents: 453a66c
Author: stack <st...@apache.org>
Authored: Sat Dec 12 19:34:23 2015 -0800
Committer: stack <st...@apache.org>
Committed: Sat Dec 12 19:34:23 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f34d3e1d/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
index dbf2f0d..1ddcab8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
@@ -160,7 +160,7 @@ public class TestFlushSnapshotFromClient {
    * Test snapshotting a table that is online without flushing
    * @throws Exception
    */
-  @Ignore ("Flakey test") @Test(timeout=30000)
+  @Test(timeout=30000)
   public void testSkipFlushTableSnapshot() throws Exception {
     Admin admin = UTIL.getHBaseAdmin();
     // make sure we don't fail on listing snapshots
@@ -169,6 +169,7 @@ public class TestFlushSnapshotFromClient {
     // put some stuff in the table
     Table table = UTIL.getConnection().getTable(TABLE_NAME);
     UTIL.loadTable(table, TEST_FAM);
+    UTIL.flush(TABLE_NAME);
 
     LOG.debug("FS state before snapshot:");
     UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);


[04/26] hbase git commit: HBASE-14851 Add test showing how to use per put TTL from thrift

Posted by sy...@apache.org.
HBASE-14851 Add test showing how to use per put TTL from thrift


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

Branch: refs/heads/hbase-12439
Commit: 9647fee3f0f196d064879afd41b9eff51d5aa036
Parents: c1e0fcc
Author: Elliott Clark <ec...@apache.org>
Authored: Thu Nov 19 14:19:00 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Dec 9 16:04:10 2015 -0800

----------------------------------------------------------------------
 .../thrift2/TestThriftHBaseServiceHandler.java  | 50 ++++++++++++++++++++
 1 file changed, 50 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9647fee3/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 8003bff..654324d 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -684,6 +684,56 @@ public class TestThriftHBaseServiceHandler {
     }
   }
 
+  @Test
+  public void testPutTTL() throws Exception {
+    ThriftHBaseServiceHandler handler = createHandler();
+    byte[] rowName = "testPutTTL".getBytes();
+    ByteBuffer table = wrap(tableAname);
+    List<TColumnValue> columnValues = new ArrayList<TColumnValue>();
+
+    // Add some dummy data
+    columnValues.add(
+        new TColumnValue(
+            wrap(familyAname),
+            wrap(qualifierAname),
+            wrap(Bytes.toBytes(1L))));
+
+
+    TPut put = new TPut(wrap(rowName), columnValues);
+    put.setColumnValues(columnValues);
+
+    Map<ByteBuffer, ByteBuffer> attributes = new HashMap<>();
+
+    // Time in ms for the kv's to live.
+    long ttlTimeMs = 2000L;
+
+    // the _ttl attribute is a number of ms ttl for key values in this put.
+    attributes.put(wrap(Bytes.toBytes("_ttl")), wrap(Bytes.toBytes(ttlTimeMs)));
+    // Attach the attributes
+    put.setAttributes(attributes);
+    // Send it.
+    handler.put(table, put);
+
+    // Now get the data back
+    TGet getOne = new TGet(wrap(rowName));
+    TResult resultOne = handler.get(table, getOne);
+
+    // It's there.
+    assertArrayEquals(rowName, resultOne.getRow());
+    assertEquals(1, resultOne.getColumnValuesSize());
+
+    // Sleep 30 seconds just to make 100% sure that the key value should be expired.
+    Thread.sleep(ttlTimeMs * 15);
+
+    TGet getTwo = new TGet(wrap(rowName));
+    TResult resultTwo = handler.get(table, getTwo);
+
+
+    // Nothing should be there since it's ttl'd out.
+    assertNull(resultTwo.getRow());
+    assertEquals(0, resultTwo.getColumnValuesSize());
+  }
+
   /**
    * Padding numbers to make comparison of sort order easier in a for loop
    *


[03/26] hbase git commit: HBASE-14866 VerifyReplication and ReplicationAdmin should use full peer configuration for peer connection

Posted by sy...@apache.org.
HBASE-14866 VerifyReplication and ReplicationAdmin should use full peer configuration for peer connection


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

Branch: refs/heads/hbase-12439
Commit: c1e0fcc26d7e7b10f6ce609e1ff0e4e9378dcf4b
Parents: ba3aa9a
Author: Gary Helmling <ga...@apache.org>
Authored: Wed Dec 9 15:52:27 2015 -0800
Committer: Gary Helmling <ga...@apache.org>
Committed: Wed Dec 9 15:52:27 2015 -0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    |  14 +-
 .../replication/ReplicationPeersZKImpl.java     |   7 +-
 .../replication/ReplicationStateZKBase.java     |   3 +-
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java | 155 ----------
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   | 124 --------
 .../hadoop/hbase/zookeeper/TestZKUtil.java      |  11 -
 .../apache/hadoop/hbase/HBaseConfiguration.java |  78 ++++-
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java | 301 +++++++++++++++++++
 .../hadoop/hbase/TestHBaseConfiguration.java    |  10 +-
 .../hadoop/hbase/zookeeper/TestZKConfig.java    | 126 ++++++++
 .../hadoop/hbase/mapreduce/SyncTable.java       |  15 +-
 .../hbase/mapreduce/TableMapReduceUtil.java     |  34 ++-
 .../hbase/mapreduce/TableOutputFormat.java      |  22 +-
 .../replication/VerifyReplication.java          |  25 +-
 .../hbase/util/ServerRegionReplicaUtil.java     |   4 +-
 .../org/apache/hadoop/hbase/TestZooKeeper.java  |  65 ----
 .../replication/TestReplicationAdmin.java       |  36 ++-
 .../replication/TestReplicationEndpoint.java    |  10 +-
 .../replication/TestReplicationStateBasic.java  |   4 +-
 .../replication/TestReplicationStateZKImpl.java |   5 +-
 .../TestRegionReplicaReplicationEndpoint.java   |   8 +-
 .../hadoop/hbase/zookeeper/TestZKConfig.java    |  45 ---
 22 files changed, 620 insertions(+), 482 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index 8bd1267..a0bea8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
@@ -626,7 +625,8 @@ public class ReplicationAdmin implements Closeable {
     }
   }
 
-  private List<ReplicationPeer> listValidReplicationPeers() {
+  @VisibleForTesting
+  List<ReplicationPeer> listValidReplicationPeers() {
     Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
     if (peers == null || peers.size() <= 0) {
       return null;
@@ -634,18 +634,16 @@ public class ReplicationAdmin implements Closeable {
     List<ReplicationPeer> validPeers = new ArrayList<ReplicationPeer>(peers.size());
     for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
       String peerId = peerEntry.getKey();
-      String clusterKey = peerEntry.getValue().getClusterKey();
-      Configuration peerConf = new Configuration(this.connection.getConfiguration());
       Stat s = null;
       try {
-        ZKUtil.applyClusterKeyToConf(peerConf, clusterKey);
         Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
+        Configuration peerConf = pair.getSecond();
         ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
         s =
             zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
               null);
         if (null == s) {
-          LOG.info(peerId + ' ' + clusterKey + " is invalid now.");
+          LOG.info(peerId + ' ' + pair.getFirst().getClusterKey() + " is invalid now.");
           continue;
         }
         validPeers.add(peer);
@@ -664,10 +662,6 @@ public class ReplicationAdmin implements Closeable {
         LOG.debug("Failure details to get valid replication peers.", e);
         Thread.currentThread().interrupt();
         continue;
-      } catch (IOException e) {
-        LOG.warn("Failed to get valid replication peers due to IOException.");
-        LOG.debug("Failure details to get valid replication peers.", e);
-        continue;
       }
     }
     return validPeers;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 1884469..63f9ac3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
@@ -318,11 +319,9 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       return null;
     }
 
-    Configuration otherConf = new Configuration(this.conf);
+    Configuration otherConf;
     try {
-      if (peerConfig.getClusterKey() != null && !peerConfig.getClusterKey().isEmpty()) {
-        ZKUtil.applyClusterKeyToConf(otherConf, peerConfig.getClusterKey());
-      }
+      otherConf = HBaseConfiguration.createClusterConf(this.conf, peerConfig.getClusterKey());
     } catch (IOException e) {
       LOG.error("Can't get peer configuration for peerId=" + peerId + " because:", e);
       return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 1691b3f..4fbac0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -69,7 +70,7 @@ public abstract class ReplicationStateZKBase {
     String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
     String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
     this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
-    this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf);
+    this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
     this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
     this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
     this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
deleted file mode 100644
index a8f1182..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
+++ /dev/null
@@ -1,155 +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.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.Map.Entry;
-import java.util.Properties;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Utility methods for reading, and building the ZooKeeper configuration.
- *
- * The order and priority for reading the config are as follows:
- * (1). Property with "hbase.zookeeper.property." prefix from HBase XML
- * (2). other zookeeper related properties in HBASE XML
- */
-@InterfaceAudience.Private
-public class ZKConfig {
-
-  private static final String VARIABLE_START = "${";
-
-  /**
-   * Make a Properties object holding ZooKeeper config.
-   * Parses the corresponding config options from the HBase XML configs
-   * and generates the appropriate ZooKeeper properties.
-   * @param conf Configuration to read from.
-   * @return Properties holding mappings representing ZooKeeper config file.
-   */
-  public static Properties makeZKProps(Configuration conf) {
-    return makeZKPropsFromHbaseConfig(conf);
-  }
-
-  /**
-   * Make a Properties object holding ZooKeeper config.
-   * Parses the corresponding config options from the HBase XML configs
-   * and generates the appropriate ZooKeeper properties.
-   *
-   * @param conf Configuration to read from.
-   * @return Properties holding mappings representing ZooKeeper config file.
-   */
-  private static Properties makeZKPropsFromHbaseConfig(Configuration conf) {
-    Properties zkProperties = new Properties();
-
-    // Directly map all of the hbase.zookeeper.property.KEY properties.
-    // Synchronize on conf so no loading of configs while we iterate
-    synchronized (conf) {
-      for (Entry<String, String> entry : conf) {
-        String key = entry.getKey();
-        if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
-          String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN);
-          String value = entry.getValue();
-          // If the value has variables substitutions, need to do a get.
-          if (value.contains(VARIABLE_START)) {
-            value = conf.get(key);
-          }
-          zkProperties.setProperty(zkKey, value);
-        }
-      }
-    }
-
-    // If clientPort is not set, assign the default.
-    if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) {
-      zkProperties.put(HConstants.CLIENT_PORT_STR,
-          HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
-    }
-
-    // Create the server.X properties.
-    int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
-    int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
-
-    final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
-                                                 HConstants.LOCALHOST);
-    String serverHost;
-    String address;
-    String key;
-    for (int i = 0; i < serverHosts.length; ++i) {
-      if (serverHosts[i].contains(":")) {
-        serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':'));
-      } else {
-        serverHost = serverHosts[i];
-      }
-      address = serverHost + ":" + peerPort + ":" + leaderPort;
-      key = "server." + i;
-      zkProperties.put(key, address);
-    }
-
-    return zkProperties;
-  }
-
-  /**
-   * Return the ZK Quorum servers string given the specified configuration
-   *
-   * @param conf
-   * @return Quorum servers String
-   */
-  private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) {
-    String defaultClientPort = Integer.toString(
-        conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
-
-    // Build the ZK quorum server string with "server:clientport" list, separated by ','
-    final String[] serverHosts =
-        conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
-    return buildQuorumServerString(serverHosts, defaultClientPort);
-  }
-
-  /**
-   * Build the ZK quorum server string with "server:clientport" list, separated by ','
-   *
-   * @param serverHosts a list of servers for ZK quorum
-   * @param clientPort the default client port
-   * @return the string for a list of "server:port" separated by ","
-   */
-  public static String buildQuorumServerString(String[] serverHosts, String clientPort) {
-    StringBuilder quorumStringBuilder = new StringBuilder();
-    String serverHost;
-    for (int i = 0; i < serverHosts.length; ++i) {
-      if (serverHosts[i].contains(":")) {
-        serverHost = serverHosts[i]; // just use the port specified from the input
-      } else {
-        serverHost = serverHosts[i] + ":" + clientPort;
-      }
-      if (i > 0) {
-        quorumStringBuilder.append(',');
-      }
-      quorumStringBuilder.append(serverHost);
-    }
-    return quorumStringBuilder.toString();
-  }
-
-  /**
-   * Return the ZK Quorum servers string given the specified configuration.
-   * @return Quorum servers
-   */
-  public static String getZKQuorumServersString(Configuration conf) {
-    return getZKQuorumServersStringFromHbaseConfig(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 633525f..c268268 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -76,7 +76,6 @@ import org.apache.zookeeper.proto.DeleteRequest;
 import org.apache.zookeeper.proto.SetDataRequest;
 import org.apache.zookeeper.server.ZooKeeperSaslServer;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
@@ -96,25 +95,6 @@ public class ZKUtil {
   public static final char ZNODE_PATH_SEPARATOR = '/';
   private static int zkDumpConnectionTimeOut;
 
-  // The Quorum for the ZK cluster can have one the following format (see examples below):
-  // (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort)
-  // (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
-  //      in this case, the clientPort would be ignored)
-  // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
-  //      the clientPort; otherwise, it would use the specified port)
-  @VisibleForTesting
-  public static class ZKClusterKey {
-    public String quorumString;
-    public int clientPort;
-    public String znodeParent;
-
-    ZKClusterKey(String quorumString, int clientPort, String znodeParent) {
-      this.quorumString = quorumString;
-      this.clientPort = clientPort;
-      this.znodeParent = znodeParent;
-    }
-  }
-
   /**
    * Creates a new connection to ZooKeeper, pulling settings and ensemble config
    * from the specified configuration object using methods from {@link ZKConfig}.
@@ -361,110 +341,6 @@ public class ZKUtil {
     return path.substring(path.lastIndexOf("/")+1);
   }
 
-  /**
-   * Get the key to the ZK ensemble for this configuration without
-   * adding a name at the end
-   * @param conf Configuration to use to build the key
-   * @return ensemble key without a name
-   */
-  public static String getZooKeeperClusterKey(Configuration conf) {
-    return getZooKeeperClusterKey(conf, null);
-  }
-
-  /**
-   * Get the key to the ZK ensemble for this configuration and append
-   * a name at the end
-   * @param conf Configuration to use to build the key
-   * @param name Name that should be appended at the end if not empty or null
-   * @return ensemble key with a name (if any)
-   */
-  public static String getZooKeeperClusterKey(Configuration conf, String name) {
-    String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM).replaceAll(
-        "[\\t\\n\\x0B\\f\\r]", "");
-    StringBuilder builder = new StringBuilder(ensemble);
-    builder.append(":");
-    builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
-    builder.append(":");
-    builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
-    if (name != null && !name.isEmpty()) {
-      builder.append(",");
-      builder.append(name);
-    }
-    return builder.toString();
-  }
-
-  /**
-   * Apply the settings in the given key to the given configuration, this is
-   * used to communicate with distant clusters
-   * @param conf configuration object to configure
-   * @param key string that contains the 3 required configuratins
-   * @throws IOException
-   */
-  public static void applyClusterKeyToConf(Configuration conf, String key)
-      throws IOException{
-    ZKClusterKey zkClusterKey = transformClusterKey(key);
-    conf.set(HConstants.ZOOKEEPER_QUORUM, zkClusterKey.quorumString);
-    conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClusterKey.clientPort);
-    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkClusterKey.znodeParent);
-  }
-
-  /**
-   * Separate the given key into the three configurations it should contain:
-   * hbase.zookeeper.quorum, hbase.zookeeper.client.port
-   * and zookeeper.znode.parent
-   * @param key
-   * @return the three configuration in the described order
-   * @throws IOException
-   */
-  public static ZKClusterKey transformClusterKey(String key) throws IOException {
-    String[] parts = key.split(":");
-
-    if (parts.length == 3) {
-      return new ZKClusterKey(parts [0], Integer.parseInt(parts [1]), parts [2]);
-    }
-
-    if (parts.length > 3) {
-      // The quorum could contain client port in server:clientport format, try to transform more.
-      String zNodeParent = parts [parts.length - 1];
-      String clientPort = parts [parts.length - 2];
-
-      // The first part length is the total length minus the lengths of other parts and minus 2 ":"
-      int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2;
-      String quorumStringInput = key.substring(0, endQuorumIndex);
-      String[] serverHosts = quorumStringInput.split(",");
-
-      // The common case is that every server has its own client port specified - this means
-      // that (total parts - the ZNodeParent part - the ClientPort part) is equal to
-      // (the number of "," + 1) - "+ 1" because the last server has no ",".
-      if ((parts.length - 2) == (serverHosts.length + 1)) {
-        return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent);
-      }
-
-      // For the uncommon case that some servers has no port specified, we need to build the
-      // server:clientport list using default client port for servers without specified port.
-      return new ZKClusterKey(
-        ZKConfig.buildQuorumServerString(serverHosts, clientPort),
-        Integer.parseInt(clientPort),
-        zNodeParent);
-    }
-
-    throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
-          HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
-          + HConstants.ZOOKEEPER_ZNODE_PARENT);
-  }
-
-  /**
-   * Standardize the ZK quorum string: make it a "server:clientport" list, separated by ','
-   * @param quorumStringInput a string contains a list of servers for ZK quorum
-   * @param clientPort the default client port
-   * @return the string for a list of "server:port" separated by ","
-   */
-  @VisibleForTesting
-  public static String standardizeQuorumServerString(String quorumStringInput, String clientPort) {
-    String[] serverHosts = quorumStringInput.split(",");
-    return ZKConfig.buildQuorumServerString(serverHosts, clientPort);
-  }
-
   //
   // Existence checks and watches
   //

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
index 72de935..eb629f2 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
@@ -41,17 +41,6 @@ import org.junit.experimental.categories.Category;
 public class TestZKUtil {
 
   @Test
-  public void testGetZooKeeperClusterKey() {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(HConstants.ZOOKEEPER_QUORUM, "\tlocalhost\n");
-    conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, "3333");
-    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "hbase");
-    String clusterKey = ZKUtil.getZooKeeperClusterKey(conf, "test");
-    Assert.assertTrue(!clusterKey.contains("\t") && !clusterKey.contains("\n"));
-    Assert.assertEquals("localhost:3333:hbase,test", clusterKey);
-  }
-  
-  @Test
   public void testCreateACL() throws ZooKeeperConnectionException, IOException {
     Configuration conf = HBaseConfiguration.create();
     conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3");

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
index 7a037f4..7b94c3d 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.Map.Entry;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
 import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 
 /**
  * Adds HBase configuration files to a Configuration
@@ -115,7 +116,7 @@ public class HBaseConfiguration extends Configuration {
    * @param srcConf the source configuration
    **/
   public static void merge(Configuration destConf, Configuration srcConf) {
-    for (Entry<String, String> e : srcConf) {
+    for (Map.Entry<String, String> e : srcConf) {
       destConf.set(e.getKey(), e.getValue());
     }
   }
@@ -129,7 +130,7 @@ public class HBaseConfiguration extends Configuration {
    */
   public static Configuration subset(Configuration srcConf, String prefix) {
     Configuration newConf = new Configuration(false);
-    for (Entry<String, String> entry : srcConf) {
+    for (Map.Entry<String, String> entry : srcConf) {
       if (entry.getKey().startsWith(prefix)) {
         String newKey = entry.getKey().substring(prefix.length());
         // avoid entries that would produce an empty key
@@ -142,6 +143,18 @@ public class HBaseConfiguration extends Configuration {
   }
 
   /**
+   * Sets all the entries in the provided {@code Map<String, String>} as properties in the
+   * given {@code Configuration}.  Each property will have the specified prefix prepended,
+   * so that the configuration entries are keyed by {@code prefix + entry.getKey()}.
+   */
+  public static void setWithPrefix(Configuration conf, String prefix,
+                                   Iterable<Map.Entry<String, String>> properties) {
+    for (Map.Entry<String, String> entry : properties) {
+      conf.set(prefix + entry.getKey(), entry.getValue());
+    }
+  }
+
+  /**
    * @return whether to show HBase Configuration in servlet
    */
   public static boolean isShowConfInServlet() {
@@ -236,6 +249,65 @@ public class HBaseConfiguration extends Configuration {
   }
 
   /**
+   * Generates a {@link Configuration} instance by applying the ZooKeeper cluster key
+   * to the base Configuration.  Note that additional configuration properties may be needed
+   * for a remote cluster, so it is preferable to use
+   * {@link #createClusterConf(Configuration, String, String)}.
+   *
+   * @param baseConf the base configuration to use, containing prefixed override properties
+   * @param clusterKey the ZooKeeper quorum cluster key to apply, or {@code null} if none
+   *
+   * @return the merged configuration with override properties and cluster key applied
+   *
+   * @see #createClusterConf(Configuration, String, String)
+   */
+  public static Configuration createClusterConf(Configuration baseConf, String clusterKey)
+      throws IOException {
+    return createClusterConf(baseConf, clusterKey, null);
+  }
+
+  /**
+   * Generates a {@link Configuration} instance by applying property overrides prefixed by
+   * a cluster profile key to the base Configuration.  Override properties are extracted by
+   * the {@link #subset(Configuration, String)} method, then the merged on top of the base
+   * Configuration and returned.
+   *
+   * @param baseConf the base configuration to use, containing prefixed override properties
+   * @param clusterKey the ZooKeeper quorum cluster key to apply, or {@code null} if none
+   * @param overridePrefix the property key prefix to match for override properties,
+   *     or {@code null} if none
+   * @return the merged configuration with override properties and cluster key applied
+   */
+  public static Configuration createClusterConf(Configuration baseConf, String clusterKey,
+                                                String overridePrefix) throws IOException {
+    Configuration clusterConf = HBaseConfiguration.create(baseConf);
+    if (clusterKey != null && !clusterKey.isEmpty()) {
+      applyClusterKeyToConf(clusterConf, clusterKey);
+    }
+
+    if (overridePrefix != null && !overridePrefix.isEmpty()) {
+      Configuration clusterSubset = HBaseConfiguration.subset(clusterConf, overridePrefix);
+      HBaseConfiguration.merge(clusterConf, clusterSubset);
+    }
+    return clusterConf;
+  }
+
+  /**
+   * Apply the settings in the given key to the given configuration, this is
+   * used to communicate with distant clusters
+   * @param conf configuration object to configure
+   * @param key string that contains the 3 required configuratins
+   * @throws IOException
+   */
+  private static void applyClusterKeyToConf(Configuration conf, String key)
+      throws IOException{
+    ZKConfig.ZKClusterKey zkClusterKey = ZKConfig.transformClusterKey(key);
+    conf.set(HConstants.ZOOKEEPER_QUORUM, zkClusterKey.getQuorumString());
+    conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClusterKey.getClientPort());
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkClusterKey.getZnodeParent());
+  }
+
+  /**
    * For debugging.  Dump configurations to system output as xml format.
    * Master and RS configurations can also be dumped using
    * http services. e.g. "curl http://master:16010/dump"

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
new file mode 100644
index 0000000..fe7396a
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
@@ -0,0 +1,301 @@
+/**
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Utility methods for reading, and building the ZooKeeper configuration.
+ *
+ * The order and priority for reading the config are as follows:
+ * (1). Property with "hbase.zookeeper.property." prefix from HBase XML
+ * (2). other zookeeper related properties in HBASE XML
+ */
+@InterfaceAudience.Private
+public final class ZKConfig {
+
+  private static final String VARIABLE_START = "${";
+
+  private ZKConfig() {
+  }
+
+  /**
+   * Make a Properties object holding ZooKeeper config.
+   * Parses the corresponding config options from the HBase XML configs
+   * and generates the appropriate ZooKeeper properties.
+   * @param conf Configuration to read from.
+   * @return Properties holding mappings representing ZooKeeper config file.
+   */
+  public static Properties makeZKProps(Configuration conf) {
+    return makeZKPropsFromHbaseConfig(conf);
+  }
+
+  /**
+   * Make a Properties object holding ZooKeeper config.
+   * Parses the corresponding config options from the HBase XML configs
+   * and generates the appropriate ZooKeeper properties.
+   *
+   * @param conf Configuration to read from.
+   * @return Properties holding mappings representing ZooKeeper config file.
+   */
+  private static Properties makeZKPropsFromHbaseConfig(Configuration conf) {
+    Properties zkProperties = new Properties();
+
+    // Directly map all of the hbase.zookeeper.property.KEY properties.
+    // Synchronize on conf so no loading of configs while we iterate
+    synchronized (conf) {
+      for (Entry<String, String> entry : conf) {
+        String key = entry.getKey();
+        if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
+          String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN);
+          String value = entry.getValue();
+          // If the value has variables substitutions, need to do a get.
+          if (value.contains(VARIABLE_START)) {
+            value = conf.get(key);
+          }
+          zkProperties.setProperty(zkKey, value);
+        }
+      }
+    }
+
+    // If clientPort is not set, assign the default.
+    if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) {
+      zkProperties.put(HConstants.CLIENT_PORT_STR,
+          HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
+    }
+
+    // Create the server.X properties.
+    int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
+    int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
+
+    final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
+                                                 HConstants.LOCALHOST);
+    String serverHost;
+    String address;
+    String key;
+    for (int i = 0; i < serverHosts.length; ++i) {
+      if (serverHosts[i].contains(":")) {
+        serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':'));
+      } else {
+        serverHost = serverHosts[i];
+      }
+      address = serverHost + ":" + peerPort + ":" + leaderPort;
+      key = "server." + i;
+      zkProperties.put(key, address);
+    }
+
+    return zkProperties;
+  }
+
+  /**
+   * Return the ZK Quorum servers string given the specified configuration
+   *
+   * @param conf
+   * @return Quorum servers String
+   */
+  private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) {
+    String defaultClientPort = Integer.toString(
+        conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
+
+    // Build the ZK quorum server string with "server:clientport" list, separated by ','
+    final String[] serverHosts =
+        conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
+    return buildZKQuorumServerString(serverHosts, defaultClientPort);
+  }
+
+  /**
+   * Return the ZK Quorum servers string given the specified configuration.
+   * @return Quorum servers
+   */
+  public static String getZKQuorumServersString(Configuration conf) {
+    return getZKQuorumServersStringFromHbaseConfig(conf);
+  }
+
+  /**
+   * Build the ZK quorum server string with "server:clientport" list, separated by ','
+   *
+   * @param serverHosts a list of servers for ZK quorum
+   * @param clientPort the default client port
+   * @return the string for a list of "server:port" separated by ","
+   */
+  public static String buildZKQuorumServerString(String[] serverHosts, String clientPort) {
+    StringBuilder quorumStringBuilder = new StringBuilder();
+    String serverHost;
+    for (int i = 0; i < serverHosts.length; ++i) {
+      if (serverHosts[i].contains(":")) {
+        serverHost = serverHosts[i]; // just use the port specified from the input
+      } else {
+        serverHost = serverHosts[i] + ":" + clientPort;
+      }
+      if (i > 0) {
+        quorumStringBuilder.append(',');
+      }
+      quorumStringBuilder.append(serverHost);
+    }
+    return quorumStringBuilder.toString();
+  }
+
+  /**
+   * Verifies that the given key matches the expected format for a ZooKeeper cluster key.
+   * The Quorum for the ZK cluster can have one the following formats (see examples below):
+   *
+   * <ol>
+   *   <li>s1,s2,s3 (no client port in the list, the client port could be obtained from
+   *       clientPort)</li>
+   *   <li>s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
+   *       in this case, the clientPort would be ignored)</li>
+   *   <li>s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
+   *       the clientPort; otherwise, it would use the specified port)</li>
+   * </ol>
+   *
+   * @param key the cluster key to validate
+   * @throws IOException if the key could not be parsed
+   */
+  public static void validateClusterKey(String key) throws IOException {
+    transformClusterKey(key);
+  }
+
+  /**
+   * Separate the given key into the three configurations it should contain:
+   * hbase.zookeeper.quorum, hbase.zookeeper.client.port
+   * and zookeeper.znode.parent
+   * @param key
+   * @return the three configuration in the described order
+   * @throws IOException
+   */
+  public static ZKClusterKey transformClusterKey(String key) throws IOException {
+    String[] parts = key.split(":");
+
+    if (parts.length == 3) {
+      return new ZKClusterKey(parts [0], Integer.parseInt(parts [1]), parts [2]);
+    }
+
+    if (parts.length > 3) {
+      // The quorum could contain client port in server:clientport format, try to transform more.
+      String zNodeParent = parts [parts.length - 1];
+      String clientPort = parts [parts.length - 2];
+
+      // The first part length is the total length minus the lengths of other parts and minus 2 ":"
+      int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2;
+      String quorumStringInput = key.substring(0, endQuorumIndex);
+      String[] serverHosts = quorumStringInput.split(",");
+
+      // The common case is that every server has its own client port specified - this means
+      // that (total parts - the ZNodeParent part - the ClientPort part) is equal to
+      // (the number of "," + 1) - "+ 1" because the last server has no ",".
+      if ((parts.length - 2) == (serverHosts.length + 1)) {
+        return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent);
+      }
+
+      // For the uncommon case that some servers has no port specified, we need to build the
+      // server:clientport list using default client port for servers without specified port.
+      return new ZKClusterKey(
+          buildZKQuorumServerString(serverHosts, clientPort),
+          Integer.parseInt(clientPort),
+          zNodeParent);
+    }
+
+    throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
+        HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
+        + HConstants.ZOOKEEPER_ZNODE_PARENT);
+  }
+
+  /**
+   * Get the key to the ZK ensemble for this configuration without
+   * adding a name at the end
+   * @param conf Configuration to use to build the key
+   * @return ensemble key without a name
+   */
+  public static String getZooKeeperClusterKey(Configuration conf) {
+    return getZooKeeperClusterKey(conf, null);
+  }
+
+  /**
+   * Get the key to the ZK ensemble for this configuration and append
+   * a name at the end
+   * @param conf Configuration to use to build the key
+   * @param name Name that should be appended at the end if not empty or null
+   * @return ensemble key with a name (if any)
+   */
+  public static String getZooKeeperClusterKey(Configuration conf, String name) {
+    String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM).replaceAll(
+        "[\\t\\n\\x0B\\f\\r]", "");
+    StringBuilder builder = new StringBuilder(ensemble);
+    builder.append(":");
+    builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
+    builder.append(":");
+    builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
+    if (name != null && !name.isEmpty()) {
+      builder.append(",");
+      builder.append(name);
+    }
+    return builder.toString();
+  }
+
+  /**
+   * Standardize the ZK quorum string: make it a "server:clientport" list, separated by ','
+   * @param quorumStringInput a string contains a list of servers for ZK quorum
+   * @param clientPort the default client port
+   * @return the string for a list of "server:port" separated by ","
+   */
+  @VisibleForTesting
+  public static String standardizeZKQuorumServerString(String quorumStringInput,
+      String clientPort) {
+    String[] serverHosts = quorumStringInput.split(",");
+    return buildZKQuorumServerString(serverHosts, clientPort);
+  }
+
+  // The Quorum for the ZK cluster can have one the following format (see examples below):
+  // (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort)
+  // (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
+  //      in this case, the clientPort would be ignored)
+  // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
+  //      the clientPort; otherwise, it would use the specified port)
+  @VisibleForTesting
+  public static class ZKClusterKey {
+    private String quorumString;
+    private int clientPort;
+    private String znodeParent;
+
+    ZKClusterKey(String quorumString, int clientPort, String znodeParent) {
+      this.quorumString = quorumString;
+      this.clientPort = clientPort;
+      this.znodeParent = znodeParent;
+    }
+
+    public String getQuorumString() {
+      return quorumString;
+    }
+
+    public int getClientPort() {
+      return clientPort;
+    }
+
+    public String getZnodeParent() {
+      return znodeParent;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java
index c11916f..6c14ef9 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestHBaseConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -27,11 +28,13 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.List;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -73,8 +76,11 @@ public class TestHBaseConfiguration {
     String prefix = "hbase.mapred.output.";
     conf.set("hbase.security.authentication", "kerberos");
     conf.set("hbase.regionserver.kerberos.principal", "hbasesource");
-    conf.set(prefix + "hbase.regionserver.kerberos.principal", "hbasedest");
-    conf.set(prefix, "shouldbemissing");
+    HBaseConfiguration.setWithPrefix(conf, prefix,
+        ImmutableMap.of(
+            "hbase.regionserver.kerberos.principal", "hbasedest",
+            "", "shouldbemissing")
+            .entrySet());
 
     Configuration subsetConf = HBaseConfiguration.subset(conf, prefix);
     assertNull(subsetConf.get(prefix + "hbase.regionserver.kerberos.principal"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
new file mode 100644
index 0000000..7879aea
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
@@ -0,0 +1,126 @@
+/**
+ * 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.
+ */
+package org.apache.hadoop.hbase.zookeeper;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestZKConfig {
+
+  @Test
+  public void testZKConfigLoading() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    // Test that we read only from the config instance
+    // (i.e. via hbase-default.xml and hbase-site.xml)
+    conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181);
+    Properties props = ZKConfig.makeZKProps(conf);
+    assertEquals("Property client port should have been default from the HBase config",
+                        "2181",
+                        props.getProperty("clientPort"));
+  }
+
+  @Test
+  public void testGetZooKeeperClusterKey() {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(HConstants.ZOOKEEPER_QUORUM, "\tlocalhost\n");
+    conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, "3333");
+    conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "hbase");
+    String clusterKey = ZKConfig.getZooKeeperClusterKey(conf, "test");
+    assertTrue(!clusterKey.contains("\t") && !clusterKey.contains("\n"));
+    assertEquals("localhost:3333:hbase,test", clusterKey);
+  }
+
+  @Test
+  public void testClusterKey() throws Exception {
+    testKey("server", 2181, "hbase");
+    testKey("server1,server2,server3", 2181, "hbase");
+    try {
+      ZKConfig.validateClusterKey("2181:hbase");
+    } catch (IOException ex) {
+      // OK
+    }
+  }
+
+  @Test
+  public void testClusterKeyWithMultiplePorts() throws Exception {
+    // server has different port than the default port
+    testKey("server1:2182", 2181, "hbase", true);
+    // multiple servers have their own port
+    testKey("server1:2182,server2:2183,server3:2184", 2181, "hbase", true);
+    // one server has no specified port, should use default port
+    testKey("server1:2182,server2,server3:2184", 2181, "hbase", true);
+    // the last server has no specified port, should use default port
+    testKey("server1:2182,server2:2183,server3", 2181, "hbase", true);
+    // multiple servers have no specified port, should use default port for those servers
+    testKey("server1:2182,server2,server3:2184,server4", 2181, "hbase", true);
+    // same server, different ports
+    testKey("server1:2182,server1:2183,server1", 2181, "hbase", true);
+    // mix of same server/different port and different server
+    testKey("server1:2182,server2:2183,server1", 2181, "hbase", true);
+  }
+
+  private void testKey(String ensemble, int port, String znode)
+      throws IOException {
+    testKey(ensemble, port, znode, false); // not support multiple client ports
+  }
+
+  private void testKey(String ensemble, int port, String znode, Boolean multiplePortSupport)
+      throws IOException {
+    Configuration conf = new Configuration();
+    String key = ensemble+":"+port+":"+znode;
+    String ensemble2 = null;
+    ZKConfig.ZKClusterKey zkClusterKey = ZKConfig.transformClusterKey(key);
+    if (multiplePortSupport) {
+      ensemble2 = ZKConfig.standardizeZKQuorumServerString(ensemble,
+          Integer.toString(port));
+      assertEquals(ensemble2, zkClusterKey.getQuorumString());
+    }
+    else {
+      assertEquals(ensemble, zkClusterKey.getQuorumString());
+    }
+    assertEquals(port, zkClusterKey.getClientPort());
+    assertEquals(znode, zkClusterKey.getZnodeParent());
+
+    conf = HBaseConfiguration.createClusterConf(conf, key);
+    assertEquals(zkClusterKey.getQuorumString(), conf.get(HConstants.ZOOKEEPER_QUORUM));
+    assertEquals(zkClusterKey.getClientPort(), conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, -1));
+    assertEquals(zkClusterKey.getZnodeParent(), conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
+
+    String reconstructedKey = ZKConfig.getZooKeeperClusterKey(conf);
+    if (multiplePortSupport) {
+      String key2 = ensemble2 + ":" + port + ":" + znode;
+      assertEquals(key2, reconstructedKey);
+    }
+    else {
+      assertEquals(key, reconstructedKey);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
index 20d6e24..1658ba4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/SyncTable.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
@@ -174,8 +173,9 @@ public class SyncTable extends Configured implements Tool {
       
       Configuration conf = context.getConfiguration();
       sourceHashDir = new Path(conf.get(SOURCE_HASH_DIR_CONF_KEY));
-      sourceConnection = openConnection(conf, SOURCE_ZK_CLUSTER_CONF_KEY);
-      targetConnection = openConnection(conf, TARGET_ZK_CLUSTER_CONF_KEY);
+      sourceConnection = openConnection(conf, SOURCE_ZK_CLUSTER_CONF_KEY, null);
+      targetConnection = openConnection(conf, TARGET_ZK_CLUSTER_CONF_KEY,
+          TableOutputFormat.OUTPUT_CONF_PREFIX);
       sourceTable = openTable(sourceConnection, conf, SOURCE_TABLE_CONF_KEY);
       targetTable = openTable(targetConnection, conf, TARGET_TABLE_CONF_KEY);
       dryRun = conf.getBoolean(SOURCE_TABLE_CONF_KEY, false);
@@ -196,13 +196,12 @@ public class SyncTable extends Configured implements Tool {
       targetHasher = new HashTable.ResultHasher();
     }
   
-    private static Connection openConnection(Configuration conf, String zkClusterConfKey)
+    private static Connection openConnection(Configuration conf, String zkClusterConfKey,
+                                             String configPrefix)
       throws IOException {
-        Configuration clusterConf = new Configuration(conf);
         String zkCluster = conf.get(zkClusterConfKey);
-        if (zkCluster != null) {
-          ZKUtil.applyClusterKeyToConf(clusterConf, zkCluster);
-        }
+        Configuration clusterConf = HBaseConfiguration.createClusterConf(conf,
+            zkCluster, configPrefix);
         return ConnectionFactory.createConnection(clusterConf);
     }
     

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index cc8a35c..a48871f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -43,12 +43,11 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.StringUtils;
-import com.google.protobuf.InvalidProtocolBufferException;
 
 import java.io.File;
 import java.io.IOException;
@@ -485,12 +484,8 @@ public class TableMapReduceUtil {
         String quorumAddress = job.getConfiguration().get(TableOutputFormat.QUORUM_ADDRESS);
         User user = userProvider.getCurrent();
         if (quorumAddress != null) {
-          Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
-          ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
-          // apply any "hbase.mapred.output." configuration overrides
-          Configuration outputOverrides =
-              HBaseConfiguration.subset(peerConf, TableOutputFormat.OUTPUT_CONF_PREFIX);
-          HBaseConfiguration.merge(peerConf, outputOverrides);
+          Configuration peerConf = HBaseConfiguration.createClusterConf(job.getConfiguration(),
+              quorumAddress, TableOutputFormat.OUTPUT_CONF_PREFIX);
           Connection peerConn = ConnectionFactory.createConnection(peerConf);
           try {
             TokenUtil.addTokenForJob(peerConn, user, job);
@@ -523,15 +518,30 @@ public class TableMapReduceUtil {
    * @param job The job that requires the permission.
    * @param quorumAddress string that contains the 3 required configuratins
    * @throws IOException When the authentication token cannot be obtained.
+   * @deprecated Since 1.2.0, use {@link #initCredentialsForCluster(Job, Configuration)} instead.
    */
+  @Deprecated
   public static void initCredentialsForCluster(Job job, String quorumAddress)
       throws IOException {
+    Configuration peerConf = HBaseConfiguration.createClusterConf(job.getConfiguration(),
+        quorumAddress);
+    initCredentialsForCluster(job, peerConf);
+  }
+
+  /**
+   * Obtain an authentication token, for the specified cluster, on behalf of the current user
+   * and add it to the credentials for the given map reduce job.
+   *
+   * @param job The job that requires the permission.
+   * @param conf The configuration to use in connecting to the peer cluster
+   * @throws IOException When the authentication token cannot be obtained.
+   */
+  public static void initCredentialsForCluster(Job job, Configuration conf)
+      throws IOException {
     UserProvider userProvider = UserProvider.instantiate(job.getConfiguration());
     if (userProvider.isHBaseSecurityEnabled()) {
       try {
-        Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
-        ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
-        Connection peerConn = ConnectionFactory.createConnection(peerConf);
+        Connection peerConn = ConnectionFactory.createConnection(conf);
         try {
           TokenUtil.addTokenForJob(peerConn, userProvider.getCurrent(), job);
         } finally {
@@ -680,7 +690,7 @@ public class TableMapReduceUtil {
     // If passed a quorum/ensemble address, pass it on to TableOutputFormat.
     if (quorumAddress != null) {
       // Calling this will validate the format
-      ZKUtil.transformClusterKey(quorumAddress);
+      ZKConfig.validateClusterKey(quorumAddress);
       conf.set(TableOutputFormat.QUORUM_ADDRESS,quorumAddress);
     }
     if (serverClass != null && serverImpl != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 201e78f..998d700 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
@@ -195,22 +194,19 @@ implements Configurable {
 
   @Override
   public void setConf(Configuration otherConf) {
-    this.conf = HBaseConfiguration.create(otherConf);
-
-    String tableName = this.conf.get(OUTPUT_TABLE);
+    String tableName = otherConf.get(OUTPUT_TABLE);
     if(tableName == null || tableName.length() <= 0) {
       throw new IllegalArgumentException("Must specify table name");
     }
 
-    String address = this.conf.get(QUORUM_ADDRESS);
-    int zkClientPort = this.conf.getInt(QUORUM_PORT, 0);
-    String serverClass = this.conf.get(REGION_SERVER_CLASS);
-    String serverImpl = this.conf.get(REGION_SERVER_IMPL);
+    String address = otherConf.get(QUORUM_ADDRESS);
+    int zkClientPort = otherConf.getInt(QUORUM_PORT, 0);
+    String serverClass = otherConf.get(REGION_SERVER_CLASS);
+    String serverImpl = otherConf.get(REGION_SERVER_IMPL);
 
     try {
-      if (address != null) {
-        ZKUtil.applyClusterKeyToConf(this.conf, address);
-      }
+      this.conf = HBaseConfiguration.createClusterConf(otherConf, address, OUTPUT_CONF_PREFIX);
+
       if (serverClass != null) {
         this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
       }
@@ -221,9 +217,5 @@ implements Configurable {
       LOG.error(e);
       throw new RuntimeException(e);
     }
-
-    // finally apply any remaining "hbase.mapred.output." configuration overrides
-    Configuration outputOverrides = HBaseConfiguration.subset(otherConf, OUTPUT_CONF_PREFIX);
-    HBaseConfiguration.merge(this.conf, outputOverrides);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 76ac541..e6b4802 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
@@ -70,6 +69,7 @@ public class VerifyReplication extends Configured implements Tool {
       LogFactory.getLog(VerifyReplication.class);
 
   public final static String NAME = "verifyrep";
+  private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
   static long startTime = 0;
   static long endTime = Long.MAX_VALUE;
   static int versions = -1;
@@ -130,8 +130,8 @@ public class VerifyReplication extends Configured implements Tool {
         final TableSplit tableSplit = (TableSplit)(context.getInputSplit());
 
         String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
-        Configuration peerConf = HBaseConfiguration.create(conf);
-        ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
+        Configuration peerConf = HBaseConfiguration.createClusterConf(conf,
+            zkClusterKey, PEER_CONFIG_PREFIX);
 
         TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
         connection = ConnectionFactory.createConnection(peerConf);
@@ -211,7 +211,8 @@ public class VerifyReplication extends Configured implements Tool {
     }
   }
 
-  private static String getPeerQuorumAddress(final Configuration conf) throws IOException {
+  private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
+      final Configuration conf) throws IOException {
     ZooKeeperWatcher localZKW = null;
     ReplicationPeerZKImpl peer = null;
     try {
@@ -228,8 +229,8 @@ public class VerifyReplication extends Configured implements Tool {
       if (pair == null) {
         throw new IOException("Couldn't get peer conf!");
       }
-      Configuration peerConf = rp.getPeerConf(peerId).getSecond();
-      return ZKUtil.getZooKeeperClusterKey(peerConf);
+
+      return pair;
     } catch (ReplicationException e) {
       throw new IOException(
           "An error occured while trying to connect to the remove peer cluster", e);
@@ -268,9 +269,14 @@ public class VerifyReplication extends Configured implements Tool {
       conf.set(NAME+".families", families);
     }
 
-    String peerQuorumAddress = getPeerQuorumAddress(conf);
+    Pair<ReplicationPeerConfig, Configuration> peerConfigPair = getPeerQuorumConfig(conf);
+    ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();
+    String peerQuorumAddress = peerConfig.getClusterKey();
+    LOG.info("Peer Quorum Address: " + peerQuorumAddress + ", Peer Configuration: " +
+        peerConfig.getConfiguration());
     conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);
-    LOG.info("Peer Quorum Address: " + peerQuorumAddress);
+    HBaseConfiguration.setWithPrefix(conf, PEER_CONFIG_PREFIX,
+        peerConfig.getConfiguration().entrySet());
 
     conf.setInt(NAME + ".versions", versions);
     LOG.info("Number of version: " + versions);
@@ -293,8 +299,9 @@ public class VerifyReplication extends Configured implements Tool {
     TableMapReduceUtil.initTableMapperJob(tableName, scan,
         Verifier.class, null, null, job);
 
+    Configuration peerClusterConf = peerConfigPair.getSecond();
     // Obtain the auth token from peer cluster
-    TableMapReduceUtil.initCredentialsForCluster(job, peerQuorumAddress);
+    TableMapReduceUtil.initCredentialsForCluster(job, peerClusterConf);
 
     job.setOutputFormatClass(NullOutputFormat.class);
     job.setNumReduceTasks(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
index 5c61afb..2ba1b47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerRegionReplicaUtil.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 
 /**
  * Similar to {@link RegionReplicaUtil} but for the server side
@@ -148,7 +148,7 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
     try {
       if (repAdmin.getPeerConfig(REGION_REPLICA_REPLICATION_PEER) == null) {
         ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
-        peerConfig.setClusterKey(ZKUtil.getZooKeeperClusterKey(conf));
+        peerConfig.setClusterKey(ZKConfig.getZooKeeperClusterKey(conf));
         peerConfig.setReplicationEndpointImpl(RegionReplicaReplicationEndpoint.class.getName());
         repAdmin.addPeer(REGION_REPLICA_REPLICATION_PEER, peerConfig, null);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
index 4e1599a..77d01e2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
@@ -343,71 +343,6 @@ public class TestZooKeeper {
     assertNull(ZKUtil.getDataNoWatch(zkw, "/l1/l2", null));
   }
 
-  @Test
-  public void testClusterKey() throws Exception {
-    testKey("server", 2181, "hbase");
-    testKey("server1,server2,server3", 2181, "hbase");
-    try {
-      ZKUtil.transformClusterKey("2181:hbase");
-    } catch (IOException ex) {
-      // OK
-    }
-  }
-
-  @Test
-  public void testClusterKeyWithMultiplePorts() throws Exception {
-    // server has different port than the default port
-    testKey("server1:2182", 2181, "hbase", true);
-    // multiple servers have their own port
-    testKey("server1:2182,server2:2183,server3:2184", 2181, "hbase", true);
-    // one server has no specified port, should use default port
-    testKey("server1:2182,server2,server3:2184", 2181, "hbase", true);
-    // the last server has no specified port, should use default port
-    testKey("server1:2182,server2:2183,server3", 2181, "hbase", true);
-    // multiple servers have no specified port, should use default port for those servers
-    testKey("server1:2182,server2,server3:2184,server4", 2181, "hbase", true);
-    // same server, different ports
-    testKey("server1:2182,server1:2183,server1", 2181, "hbase", true);
-    // mix of same server/different port and different server
-    testKey("server1:2182,server2:2183,server1", 2181, "hbase", true);
-  }
-
-  private void testKey(String ensemble, int port, String znode)
-      throws IOException {
-    testKey(ensemble, port, znode, false); // not support multiple client ports
-  }
-
-  private void testKey(String ensemble, int port, String znode, Boolean multiplePortSupport)
-      throws IOException {
-    Configuration conf = new Configuration();
-    String key = ensemble+":"+port+":"+znode;
-    String ensemble2 = null;
-    ZKUtil.ZKClusterKey zkClusterKey = ZKUtil.transformClusterKey(key);
-    if (multiplePortSupport) {
-      ensemble2 = ZKUtil.standardizeQuorumServerString(ensemble, Integer.toString(port));
-      assertEquals(ensemble2, zkClusterKey.quorumString);
-    }
-    else {
-      assertEquals(ensemble, zkClusterKey.quorumString);
-    }
-    assertEquals(port, zkClusterKey.clientPort);
-    assertEquals(znode, zkClusterKey.znodeParent);
-
-    ZKUtil.applyClusterKeyToConf(conf, key);
-    assertEquals(zkClusterKey.quorumString, conf.get(HConstants.ZOOKEEPER_QUORUM));
-    assertEquals(zkClusterKey.clientPort, conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, -1));
-    assertEquals(zkClusterKey.znodeParent, conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
-
-    String reconstructedKey = ZKUtil.getZooKeeperClusterKey(conf);
-    if (multiplePortSupport) {
-      String key2 = ensemble2 + ":" + port + ":" + znode;
-      assertEquals(key2, reconstructedKey);
-    }
-    else {
-      assertEquals(key, reconstructedKey);
-    }
-  }
-
   /**
    * A test for HBASE-3238
    * @throws IOException A connection attempt to zk failed

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index e187b9b..e18220d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -24,9 +24,13 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -38,10 +42,12 @@ import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
-import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 
 /**
  * Unit testing of ReplicationAdmin
@@ -117,7 +123,29 @@ public class TestReplicationAdmin {
     admin.removePeer(ID_SECOND);
     assertEquals(0, admin.getPeersCount());
   }
-  
+
+  /**
+   * Tests that the peer configuration used by ReplicationAdmin contains all
+   * the peer's properties.
+   */
+  @Test
+  public void testPeerConfig() throws Exception {
+    ReplicationPeerConfig config = new ReplicationPeerConfig();
+    config.setClusterKey(KEY_ONE);
+    config.getConfiguration().put("key1", "value1");
+    config.getConfiguration().put("key2", "value2");
+    admin.addPeer(ID_ONE, config, null);
+
+    List<ReplicationPeer> peers = admin.listValidReplicationPeers();
+    assertEquals(1, peers.size());
+    ReplicationPeer peerOne = peers.get(0);
+    assertNotNull(peerOne);
+    assertEquals("value1", peerOne.getConfiguration().get("key1"));
+    assertEquals("value2", peerOne.getConfiguration().get("key2"));
+
+    admin.removePeer(ID_ONE);
+  }
+
   @Test
   public void testAddPeerWithUnDeletedQueues() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
index 52fb41c..a5a4e73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEndpoint.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -115,7 +115,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
   public void testCustomReplicationEndpoint() throws Exception {
     // test installing a custom replication endpoint other than the default one.
     admin.addPeer("testCustomReplicationEndpoint",
-      new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
+      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
         .setReplicationEndpointImpl(ReplicationEndpointForTest.class.getName()), null);
 
     // check whether the class has been constructed and started
@@ -157,7 +157,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     int peerCount = admin.getPeersCount();
     final String id = "testReplicationEndpointReturnsFalseOnReplicate";
     admin.addPeer(id,
-      new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
+      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
         .setReplicationEndpointImpl(ReplicationEndpointReturningFalse.class.getName()), null);
     // This test is flakey and then there is so much stuff flying around in here its, hard to
     // debug.  Peer needs to be up for the edit to make it across. This wait on
@@ -209,7 +209,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
     }
 
     admin.addPeer(id,
-        new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf2))
+        new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf2))
             .setReplicationEndpointImpl(InterClusterReplicationEndpointForTest.class.getName()),
         null);
 
@@ -234,7 +234,7 @@ public class TestReplicationEndpoint extends TestReplicationBase {
   @Test (timeout=120000)
   public void testWALEntryFilterFromReplicationEndpoint() throws Exception {
     admin.addPeer("testWALEntryFilterFromReplicationEndpoint",
-      new ReplicationPeerConfig().setClusterKey(ZKUtil.getZooKeeperClusterKey(conf1))
+      new ReplicationPeerConfig().setClusterKey(ZKConfig.getZooKeeperClusterKey(conf1))
         .setReplicationEndpointImpl(ReplicationEndpointWithWALEntryFilter.class.getName()), null);
     // now replicate some data.
     try (Connection connection = ConnectionFactory.createConnection(conf1)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index f05eceb..696c130 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -27,7 +27,7 @@ import java.util.SortedSet;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Before;
 import org.junit.Test;
@@ -202,7 +202,7 @@ public abstract class TestReplicationStateBasic {
       fail("There are no connected peers, should have thrown an IllegalArgumentException");
     } catch (IllegalArgumentException e) {
     }
-    assertEquals(KEY_ONE, ZKUtil.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
     rp.removePeer(ID_ONE);
     rp.peerRemoved(ID_ONE);
     assertNumberOfPeers(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index fff6c9d..4587c61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
@@ -79,7 +80,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     String fakeRs = ZKUtil.joinZNode(zkw1.rsZNode, "hostname1.example.org:1234");
     ZKUtil.createWithParents(zkw1, fakeRs);
     ZKClusterId.setClusterId(zkw1, new ClusterId());
-    return ZKUtil.getZooKeeperClusterKey(testConf);
+    return ZKConfig.getZooKeeperClusterKey(testConf);
   }
 
   @Before
@@ -94,7 +95,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     rq3 = ReplicationFactory.getReplicationQueues(zkw, conf, ds3);
     rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, ds1);
     rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
-    OUR_KEY = ZKUtil.getZooKeeperClusterKey(conf);
+    OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
     rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 2231f0e..65600ff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -53,7 +53,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -129,7 +129,8 @@ public class TestRegionReplicaReplicationEndpoint {
     // assert peer configuration is correct
     peerConfig = admin.getPeerConfig(peerId);
     assertNotNull(peerConfig);
-    assertEquals(peerConfig.getClusterKey(), ZKUtil.getZooKeeperClusterKey(HTU.getConfiguration()));
+    assertEquals(peerConfig.getClusterKey(), ZKConfig.getZooKeeperClusterKey(
+        HTU.getConfiguration()));
     assertEquals(peerConfig.getReplicationEndpointImpl(),
       RegionReplicaReplicationEndpoint.class.getName());
     admin.close();
@@ -162,7 +163,8 @@ public class TestRegionReplicaReplicationEndpoint {
     // assert peer configuration is correct
     peerConfig = admin.getPeerConfig(peerId);
     assertNotNull(peerConfig);
-    assertEquals(peerConfig.getClusterKey(), ZKUtil.getZooKeeperClusterKey(HTU.getConfiguration()));
+    assertEquals(peerConfig.getClusterKey(), ZKConfig.getZooKeeperClusterKey(
+        HTU.getConfiguration()));
     assertEquals(peerConfig.getReplicationEndpointImpl(),
       RegionReplicaReplicationEndpoint.class.getName());
     admin.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c1e0fcc2/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
deleted file mode 100644
index 8f5961f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
+++ /dev/null
@@ -1,45 +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.
- */
-package org.apache.hadoop.hbase.zookeeper;
-
-import java.util.Properties;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestZKConfig {
-
-  @Test
-  public void testZKConfigLoading() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    // Test that we read only from the config instance
-    // (i.e. via hbase-default.xml and hbase-site.xml)
-    conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181);
-    Properties props = ZKConfig.makeZKProps(conf);
-    Assert.assertEquals("Property client port should have been default from the HBase config",
-                        "2181",
-                        props.getProperty("clientPort"));
-  }
-}


[11/26] hbase git commit: HBASE-14769 Remove unused functions and duplicate javadocs from HBaseAdmin

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 66079dd..7a50458 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -221,7 +221,7 @@ public class HBaseAdmin implements Admin {
     return operationTimeout;
   }
 
-  HBaseAdmin(ClusterConnection connection) {
+  HBaseAdmin(ClusterConnection connection) throws IOException {
     this.conf = connection.getConfiguration();
     this.connection = connection;
 
@@ -253,13 +253,6 @@ public class HBaseAdmin implements Admin {
     return this.aborted;
   }
 
-  /**
-   * Abort a procedure
-   * @param procId ID of the procedure to abort
-   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
-   * @return true if aborted, false if procedure already completed or does not exist
-   * @throws IOException
-   */
   @Override
   public boolean abortProcedure(
       final long procId,
@@ -280,18 +273,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Abort a procedure but does not block and wait for it be completely removed.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param procId ID of the procedure to abort
-   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
-   * @return true if aborted, false if procedure already completed or does not exist
-   * @throws IOException
-   */
   @Override
   public Future<Boolean> abortProcedureAsync(
     final long procId,
@@ -339,23 +320,6 @@ public class HBaseAdmin implements Admin {
     return connection;
   }
 
-  /** @return - true if the master server is running. Throws an exception
-   *  otherwise.
-   * @throws ZooKeeperConnectionException
-   * @throws MasterNotRunningException
-   * @deprecated this has been deprecated without a replacement
-   */
-  @Deprecated
-  public boolean isMasterRunning()
-  throws MasterNotRunningException, ZooKeeperConnectionException {
-    return connection.isMasterRunning();
-  }
-
-  /**
-   * @param tableName Table to check.
-   * @return True if table exists already.
-   * @throws IOException
-   */
   @Override
   public boolean tableExists(final TableName tableName) throws IOException {
     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
@@ -366,16 +330,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  public boolean tableExists(final byte[] tableName)
-  throws IOException {
-    return tableExists(TableName.valueOf(tableName));
-  }
-
-  public boolean tableExists(final String tableName)
-  throws IOException {
-    return tableExists(TableName.valueOf(tableName));
-  }
-
   @Override
   public HTableDescriptor[] listTables() throws IOException {
     return listTables((Pattern)null, false);
@@ -410,51 +364,6 @@ public class HBaseAdmin implements Admin {
     return listTables(Pattern.compile(regex), includeSysTables);
   }
 
-  /**
-   * List all of the names of userspace tables.
-   * @return String[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Use {@link Admin#listTableNames()} instead
-   */
-  @Deprecated
-  public String[] getTableNames() throws IOException {
-    TableName[] tableNames = listTableNames();
-    String[] result = new String[tableNames.length];
-    for (int i = 0; i < tableNames.length; i++) {
-      result[i] = tableNames[i].getNameAsString();
-    }
-    return result;
-  }
-
-  /**
-   * List all of the names of userspace tables matching the given regular expression.
-   * @param pattern The regular expression to match against
-   * @return String[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
-   */
-  @Deprecated
-  public String[] getTableNames(Pattern pattern) throws IOException {
-    TableName[] tableNames = listTableNames(pattern);
-    String[] result = new String[tableNames.length];
-    for (int i = 0; i < tableNames.length; i++) {
-      result[i] = tableNames[i].getNameAsString();
-    }
-    return result;
-  }
-
-  /**
-   * List all of the names of userspace tables matching the given regular expression.
-   * @param regex The regular expression to match against
-   * @return String[] table names
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated Use {@link Admin#listTableNames(Pattern)} instead.
-   */
-  @Deprecated
-  public String[] getTableNames(String regex) throws IOException {
-    return getTableNames(Pattern.compile(regex));
-  }
-
   @Override
   public TableName[] listTableNames() throws IOException {
     return listTableNames((Pattern)null, false);
@@ -490,23 +399,13 @@ public class HBaseAdmin implements Admin {
     return listTableNames(Pattern.compile(regex), includeSysTables);
   }
 
-  /**
-   * Method for getting the tableDescriptor
-   * @param tableName as a byte []
-   * @return the tableDescriptor
-   * @throws TableNotFoundException
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
-  public HTableDescriptor getTableDescriptor(final TableName tableName)
-  throws TableNotFoundException, IOException {
+  public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
      return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout);
   }
 
-  static HTableDescriptor getTableDescriptor(final TableName tableName,
-         HConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
-         int operationTimeout) throws TableNotFoundException, IOException {
-
+  static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
+      RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException {
       if (tableName == null) return null;
       HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
         @Override
@@ -528,11 +427,6 @@ public class HBaseAdmin implements Admin {
       throw new TableNotFoundException(tableName.getNameAsString());
   }
 
-  public HTableDescriptor getTableDescriptor(final byte[] tableName)
-  throws TableNotFoundException, IOException {
-    return getTableDescriptor(TableName.valueOf(tableName));
-  }
-
   private long getPauseTime(int tries) {
     int triesCount = tries;
     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
@@ -541,49 +435,12 @@ public class HBaseAdmin implements Admin {
     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
   }
 
-  /**
-   * Creates a new table.
-   * Synchronous operation.
-   *
-   * @param desc table descriptor for table
-   *
-   * @throws IllegalArgumentException if the table name is reserved
-   * @throws MasterNotRunningException if master is not running
-   * @throws TableExistsException if table already exists (If concurrent
-   * threads, the table may have been created between test-for-existence
-   * and attempt-at-creation).
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void createTable(HTableDescriptor desc)
   throws IOException {
     createTable(desc, null);
   }
 
-  /**
-   * Creates a new table with the specified number of regions.  The start key
-   * specified will become the end key of the first region of the table, and
-   * the end key specified will become the start key of the last region of the
-   * table (the first region has a null start key and the last region has a
-   * null end key).
-   *
-   * BigInteger math will be used to divide the key range specified into
-   * enough segments to make the required number of total regions.
-   *
-   * Synchronous operation.
-   *
-   * @param desc table descriptor for table
-   * @param startKey beginning of key range
-   * @param endKey end of key range
-   * @param numRegions the total number of regions to create
-   *
-   * @throws IllegalArgumentException if the table name is reserved
-   * @throws MasterNotRunningException if master is not running
-   * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
-   * threads, the table may have been created between test-for-existence
-   * and attempt-at-creation).
-   * @throws IOException
-   */
   @Override
   public void createTable(HTableDescriptor desc, byte [] startKey,
       byte [] endKey, int numRegions)
@@ -604,23 +461,6 @@ public class HBaseAdmin implements Admin {
     createTable(desc, splitKeys);
   }
 
-  /**
-   * Creates a new table with an initial set of empty regions defined by the
-   * specified split keys.  The total number of regions created will be the
-   * number of split keys plus one. Synchronous operation.
-   * Note : Avoid passing empty split key.
-   *
-   * @param desc table descriptor for table
-   * @param splitKeys array of split keys for the initial regions of the table
-   *
-   * @throws IllegalArgumentException if the table name is reserved, if the split keys
-   * are repeated and if the split key has empty byte array.
-   * @throws MasterNotRunningException if master is not running
-   * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
-   * threads, the table may have been created between test-for-existence
-   * and attempt-at-creation).
-   * @throws IOException
-   */
   @Override
   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
       throws IOException {
@@ -642,21 +482,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Creates a new table but does not block and wait for it to come online.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param desc table descriptor for table
-   * @param splitKeys keys to check if the table has been created with all split keys
-   * @throws IllegalArgumentException Bad table name, if the split keys
-   *    are repeated and if the split key has empty byte array.
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async creation. You can use Future.get(long, TimeUnit)
-   *    to wait on the operation to complete.
-   */
   @Override
   public Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
       throws IOException {
@@ -723,21 +548,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  public void deleteTable(final String tableName) throws IOException {
-    deleteTable(TableName.valueOf(tableName));
-  }
-
-  public void deleteTable(final byte[] tableName) throws IOException {
-    deleteTable(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Deletes a table.
-   * Synchronous operation.
-   *
-   * @param tableName name of table to delete
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteTable(final TableName tableName) throws IOException {
     Future<Void> future = deleteTableAsync(tableName);
@@ -756,18 +566,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Deletes the table but does not block and wait for it be completely removed.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param tableName name of table to delete
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async delete. You can use Future.get(long, TimeUnit)
-   *    to wait on the operation to complete.
-   */
   @Override
   public Future<Void> deleteTableAsync(final TableName tableName) throws IOException {
     DeleteTableResponse response = executeCallable(
@@ -810,19 +608,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Deletes tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.lang.String)} and
-   * {@link #deleteTable(byte[])}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
-   * @throws IOException
-   * @see #deleteTables(java.util.regex.Pattern)
-   * @see #deleteTable(java.lang.String)
-   */
   @Override
   public HTableDescriptor[] deleteTables(String regex) throws IOException {
     return deleteTables(Pattern.compile(regex));
@@ -833,7 +618,7 @@ public class HBaseAdmin implements Admin {
    *
    * Warning: Use this method carefully, there is no prompting and the effect is
    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #deleteTable(byte[])}
+   * {@link #deleteTable(TableName)}
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted
@@ -853,12 +638,6 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
-  /**
-   * Truncate a table. Synchronous operation.
-   * @param tableName name of table to truncate
-   * @param preserveSplits True if the splits should be preserved
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void truncateTable(final TableName tableName, final boolean preserveSplits)
       throws IOException {
@@ -879,18 +658,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Truncate the table but does not block and wait for it be completely enabled. You can use
-   * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
-   * ExecutionException if there was an error while executing the operation or TimeoutException in
-   * case the wait timeout was not long enough to allow the operation to complete. Asynchronous
-   * operation.
-   * @param tableName name of table to delete
-   * @param preserveSplits true if the splits should be preserved
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
-   *         operation to complete.
-   */
   @Override
   public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
       throws IOException {
@@ -947,20 +714,6 @@ public class HBaseAdmin implements Admin {
     return splits;
   }
 
-
-  /**
-   * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
-   * and {@link #isTableEnabled(byte[])} instead.
-   * The table has to be in disabled state for it to be enabled.
-   * @param tableName name of the table
-   * @throws IOException if a remote or network exception occurs
-   * There could be couple types of IOException
-   * TableNotFoundException means the table doesn't exist.
-   * TableNotDisabledException means the table isn't in disabled state.
-   * @see #isTableEnabled(byte[])
-   * @see #disableTable(byte[])
-   * @see #enableTableAsync(byte[])
-   */
   @Override
   public void enableTable(final TableName tableName)
   throws IOException {
@@ -980,16 +733,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  public void enableTable(final byte[] tableName)
-  throws IOException {
-    enableTable(TableName.valueOf(tableName));
-  }
-
-  public void enableTable(final String tableName)
-  throws IOException {
-    enableTable(TableName.valueOf(tableName));
-  }
-
   /**
    * Wait for the table to be enabled and available
    * If enabling the table exceeds the retry period, an exception is thrown.
@@ -1031,28 +774,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  public void enableTableAsync(final byte[] tableName)
-  throws IOException {
-    enableTable(TableName.valueOf(tableName));
-  }
-
-  public void enableTableAsync(final String tableName)
-  throws IOException {
-    enableTableAsync(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Enable the table but does not block and wait for it be completely enabled.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param tableName name of table to delete
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async enable. You can use Future.get(long, TimeUnit)
-   *    to wait on the operation to complete.
-   */
   @Override
   public Future<Void> enableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
@@ -1088,33 +809,11 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Enable tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.lang.String)} and
-   * {@link #enableTable(byte[])}
-   *
-   * @param regex The regular expression to match table names against
-   * @throws IOException
-   * @see #enableTables(java.util.regex.Pattern)
-   * @see #enableTable(java.lang.String)
-   */
   @Override
   public HTableDescriptor[] enableTables(String regex) throws IOException {
     return enableTables(Pattern.compile(regex));
   }
 
-  /**
-   * Enable tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #enableTable(byte[])}
-   *
-   * @param pattern The pattern to match table names against
-   * @throws IOException
-   */
   @Override
   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
@@ -1131,25 +830,6 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
-  public void disableTableAsync(final byte[] tableName) throws IOException {
-    disableTableAsync(TableName.valueOf(tableName));
-  }
-
-  public void disableTableAsync(final String tableName) throws IOException {
-    disableTableAsync(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Disable table and wait on completion.  May timeout eventually.  Use
-   * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
-   * instead.
-   * The table has to be in enabled state for it to be disabled.
-   * @param tableName
-   * @throws IOException
-   * There could be couple types of IOException
-   * TableNotFoundException means the table doesn't exist.
-   * TableNotEnabledException means the table isn't in enabled state.
-   */
   @Override
   public void disableTable(final TableName tableName)
   throws IOException {
@@ -1169,28 +849,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  public void disableTable(final byte[] tableName)
-  throws IOException {
-    disableTable(TableName.valueOf(tableName));
-  }
-
-  public void disableTable(final String tableName)
-  throws IOException {
-    disableTable(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Disable the table but does not block and wait for it be completely disabled.
-   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param tableName name of table to delete
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async disable. You can use Future.get(long, TimeUnit)
-   *    to wait on the operation to complete.
-   */
   @Override
   public Future<Void> disableTableAsync(final TableName tableName) throws IOException {
     TableName.isLegalFullyQualifiedTableName(tableName.getName());
@@ -1220,42 +878,17 @@ public class HBaseAdmin implements Admin {
     }
 
     @Override
-    protected Void waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
+    protected Void waitOperationResult(long deadlineTs) throws IOException, TimeoutException {
       waitForTableDisabled(deadlineTs);
       return null;
     }
   }
 
-  /**
-   * Disable tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.lang.String)} and
-   * {@link #disableTable(byte[])}
-   *
-   * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   * @throws IOException
-   * @see #disableTables(java.util.regex.Pattern)
-   * @see #disableTable(java.lang.String)
-   */
   @Override
   public HTableDescriptor[] disableTables(String regex) throws IOException {
     return disableTables(Pattern.compile(regex));
   }
 
-  /**
-   * Disable tables matching the passed in pattern and wait on completion.
-   *
-   * Warning: Use this method carefully, there is no prompting and the effect is
-   * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
-   * {@link #disableTable(byte[])}
-   *
-   * @param pattern The pattern to match table names against
-   * @return Table descriptors for tables that couldn't be disabled
-   * @throws IOException
-   */
   @Override
   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
@@ -1272,24 +905,9 @@ public class HBaseAdmin implements Admin {
     return failed.toArray(new HTableDescriptor[failed.size()]);
   }
 
-  /*
-   * Checks whether table exists. If not, throws TableNotFoundException
-   * @param tableName
-   */
-  private void checkTableExistence(TableName tableName) throws IOException {
-    if (!tableExists(tableName)) {
-      throw new TableNotFoundException(tableName);
-    }
-  }
-
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public boolean isTableEnabled(final TableName tableName) throws IOException {
-    checkTableExistence(tableName);
+    checkTableExists(tableName);
     return executeCallable(new ConnectionCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException, IOException {
@@ -1301,173 +919,46 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  public boolean isTableEnabled(byte[] tableName) throws IOException {
-    return isTableEnabled(TableName.valueOf(tableName));
-  }
-
-  public boolean isTableEnabled(String tableName) throws IOException {
-    return isTableEnabled(TableName.valueOf(tableName));
-  }
-
-
-
-  /**
-   * @param tableName name of table to check
-   * @return true if table is off-line
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public boolean isTableDisabled(TableName tableName) throws IOException {
-    checkTableExistence(tableName);
+    checkTableExists(tableName);
     return connection.isTableDisabled(tableName);
   }
 
-  public boolean isTableDisabled(byte[] tableName) throws IOException {
-    return isTableDisabled(TableName.valueOf(tableName));
-  }
-
-  public boolean isTableDisabled(String tableName) throws IOException {
-    return isTableDisabled(TableName.valueOf(tableName));
-  }
-
-  /**
-   * @param tableName name of table to check
-   * @return true if all regions of the table are available
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public boolean isTableAvailable(TableName tableName) throws IOException {
     return connection.isTableAvailable(tableName);
   }
 
-  public boolean isTableAvailable(byte[] tableName) throws IOException {
-    return isTableAvailable(TableName.valueOf(tableName));
-  }
-
-  public boolean isTableAvailable(String tableName) throws IOException {
-    return isTableAvailable(TableName.valueOf(tableName));
-  }
-
-  /**
-   * Use this api to check if the table has been created with the specified number of
-   * splitkeys which was used while creating the given table.
-   * Note : If this api is used after a table's region gets splitted, the api may return
-   * false.
-   * @param tableName
-   *          name of table to check
-   * @param splitKeys
-   *          keys to check if the table has been created with all split keys
-   * @throws IOException
-   *           if a remote or network excpetion occurs
-   */
   @Override
-  public boolean isTableAvailable(TableName tableName,
-                                  byte[][] splitKeys) throws IOException {
+  public boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException {
     return connection.isTableAvailable(tableName, splitKeys);
   }
 
-  public boolean isTableAvailable(byte[] tableName,
-                                  byte[][] splitKeys) throws IOException {
-    return isTableAvailable(TableName.valueOf(tableName), splitKeys);
-  }
-
-  public boolean isTableAvailable(String tableName,
-                                  byte[][] splitKeys) throws IOException {
-    return isTableAvailable(TableName.valueOf(tableName), splitKeys);
-  }
-
-  /**
-   * Get the status of alter command - indicates how many regions have received
-   * the updated schema Asynchronous operation.
-   *
-   * @param tableName TableName instance
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the
-   *         regions that are yet to be updated Pair.getSecond() is the total number
-   *         of regions of the table
-   * @throws IOException
-   *           if a remote or network exception occurs
-   */
   @Override
-  public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
-  throws IOException {
+  public Pair<Integer, Integer> getAlterStatus(final TableName tableName) throws IOException {
     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
       @Override
       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
         GetSchemaAlterStatusRequest req = RequestConverter
             .buildGetSchemaAlterStatusRequest(tableName);
         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
-        Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
-            .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
+        Pair<Integer, Integer> pair = new Pair<>(ret.getYetToUpdateRegions(),
+            ret.getTotalRegions());
         return pair;
       }
     });
   }
 
-  /**
-   * Get the status of alter command - indicates how many regions have received
-   * the updated schema Asynchronous operation.
-   *
-   * @param tableName
-   *          name of the table to get the status of
-   * @return Pair indicating the number of regions updated Pair.getFirst() is the
-   *         regions that are yet to be updated Pair.getSecond() is the total number
-   *         of regions of the table
-   * @throws IOException
-   *           if a remote or network exception occurs
-   */
   @Override
-  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
-   throws IOException {
+  public Pair<Integer, Integer> getAlterStatus(final byte[] tableName) throws IOException {
     return getAlterStatus(TableName.valueOf(tableName));
   }
 
   /**
-   * Add a column family to an existing table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of the table to add column family to
-   * @param columnFamily column family descriptor of column family to be added
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
-   */
-  @Deprecated
-  public void addColumn(final byte[] tableName, HColumnDescriptor columnFamily)
-  throws IOException {
-    addColumnFamily(TableName.valueOf(tableName), columnFamily);
-  }
-
-  /**
-   * Add a column family to an existing table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of the table to add column family to
-   * @param columnFamily column family descriptor of column family to be added
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
-   */
-  @Deprecated
-  public void addColumn(final String tableName, HColumnDescriptor columnFamily)
-  throws IOException {
-    addColumnFamily(TableName.valueOf(tableName), columnFamily);
-  }
-
-  /**
-   * Add a column family to an existing table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of the table to add column family to
-   * @param columnFamily column family descriptor of column family to be added
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #addColumnFamily(TableName, HColumnDescriptor)}.
+   * {@inheritDoc}
+   * @deprecated Since 2.0. Will be removed in 3.0. Use
+   *     {@link #addColumnFamily(TableName, HColumnDescriptor)} instead.
    */
   @Override
   @Deprecated
@@ -1506,52 +997,9 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * Delete a column family from a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily name of column family to be deleted
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
-   */
-  @Deprecated
-  public void deleteColumn(final byte[] tableName, final String columnFamily)
-  throws IOException {
-    deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
-  }
-
-  /**
-   * Delete a column family from a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily name of column family to be deleted
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
-   */
-  @Deprecated
-  public void deleteColumn(final String tableName, final String columnFamily)
-  throws IOException {
-    deleteColumnFamily(TableName.valueOf(tableName), Bytes.toBytes(columnFamily));
-  }
-
-  /**
-   * Delete a column family from a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily name of column family to be deleted
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #deleteColumnFamily(TableName, byte[])}.
+   * {@inheritDoc}
+   * @deprecated Since 2.0. Will be removed in 3.0. Use
+   *     {@link #deleteColumnFamily(TableName, byte[])} instead.
    */
   @Override
   @Deprecated
@@ -1591,52 +1039,9 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * Modify an existing column family on a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily new column family descriptor to use
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
-   */
-  @Deprecated
-  public void modifyColumn(final String tableName, HColumnDescriptor columnFamily)
-  throws IOException {
-    modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
-  }
-
-  /**
-   * Modify an existing column family on a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily new column family descriptor to use
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
-   */
-  @Deprecated
-  public void modifyColumn(final byte[] tableName, HColumnDescriptor columnFamily)
-  throws IOException {
-    modifyColumnFamily(TableName.valueOf(tableName), columnFamily);
-  }
-
-  /**
-   * Modify an existing column family on a table.
-   * Asynchronous operation.
-   *
-   * @param tableName name of table
-   * @param columnFamily new column family descriptor to use
-   * @throws IOException if a remote or network exception occurs
-   * @deprecated As of release 2.0.0.
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-1989">HBASE-1989</a>).
-   *             This will be removed in HBase 3.0.0.
-   *             Use {@link #modifyColumnFamily(TableName, HColumnDescriptor)}.
+   * {@inheritDoc}
+   * @deprecated As of 2.0. Will be removed in 3.0. Use
+   *     {@link #modifyColumnFamily(TableName, HColumnDescriptor)} instead.
    */
   @Override
   @Deprecated
@@ -1675,33 +1080,13 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Close a region. For expert-admins.  Runs close on the regionserver.  The
-   * master will not be informed of the close.
-   * @param regionname region name to close
-   * @param serverName If supplied, we'll use this location rather than
-   * the one currently in <code>hbase:meta</code>
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
-  public void closeRegion(final String regionname, final String serverName)
-  throws IOException {
+  public void closeRegion(final String regionname, final String serverName) throws IOException {
     closeRegion(Bytes.toBytes(regionname), serverName);
   }
 
-  /**
-   * Close a region.  For expert-admins  Runs close on the regionserver.  The
-   * master will not be informed of the close.
-   * @param regionname region name to close
-   * @param serverName The servername of the regionserver.  If passed null we
-   * will use servername found in the hbase:meta table. A server name
-   * is made of host, port and startcode.  Here is an example:
-   * <code> host187.example.com,60020,1289493121758</code>
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
-  public void closeRegion(final byte [] regionname, final String serverName)
-      throws IOException {
+  public void closeRegion(final byte [] regionname, final String serverName) throws IOException {
     if (serverName != null) {
       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
       if (pair == null || pair.getFirst() == null) {
@@ -1721,27 +1106,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * For expert-admins. Runs close on the regionserver. Closes a region based on
-   * the encoded region name. The region server name is mandatory. If the
-   * servername is provided then based on the online regions in the specified
-   * regionserver the specified region will be closed. The master will not be
-   * informed of the close. Note that the regionname is the encoded regionname.
-   *
-   * @param encodedRegionName
-   *          The encoded region name; i.e. the hash that makes up the region
-   *          name suffix: e.g. if regionname is
-   *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
-   *          , then the encoded region name is:
-   *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
-   * @param serverName
-   *          The servername of the regionserver. A server name is made of host,
-   *          port and startcode. This is mandatory. Here is an example:
-   *          <code> host187.example.com,60020,1289493121758</code>
-   * @return true if the region was closed, false if not.
-   * @throws IOException
-   *           if a remote or network exception occurs
-   */
   @Override
   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
       final String serverName) throws IOException {
@@ -1766,33 +1130,19 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Close a region.  For expert-admins  Runs close on the regionserver.  The
-   * master will not be informed of the close.
-   * @param sn
-   * @param hri
-   * @throws IOException
-   */
   @Override
-  public void closeRegion(final ServerName sn, final HRegionInfo hri)
-  throws IOException {
+  public void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // Close the region without updating zk state.
     ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
   }
 
-  /**
-   * Get all the online regions on a region server.
-   */
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     return ProtobufUtil.getOnlineRegions(admin);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void flush(final TableName tableName) throws IOException {
     checkTableExists(tableName);
@@ -1804,9 +1154,6 @@ public class HBaseAdmin implements Admin {
       new HashMap<String, String>());
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void flushRegion(final byte[] regionName) throws IOException {
     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
@@ -1816,7 +1163,16 @@ public class HBaseAdmin implements Admin {
     if (regionServerPair.getSecond() == null) {
       throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
     }
-    flush(regionServerPair.getSecond(), regionServerPair.getFirst());
+    HRegionInfo hRegionInfo = regionServerPair.getFirst();
+    ServerName serverName = regionServerPair.getSecond();
+    AdminService.BlockingInterface admin = this.connection.getAdmin(serverName);
+    FlushRegionRequest request =
+        RequestConverter.buildFlushRegionRequest(hRegionInfo.getRegionName());
+    try {
+      admin.flushRegion(null, request);
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
   }
 
   private void flush(final ServerName sn, final HRegionInfo hri)
@@ -1840,9 +1196,6 @@ public class HBaseAdmin implements Admin {
     compact(tableName, null, false, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void compactRegion(final byte[] regionName)
     throws IOException {
@@ -1878,18 +1231,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void majorCompact(final TableName tableName)
   throws IOException {
     compact(tableName, null, true, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void majorCompactRegion(final byte[] regionName)
   throws IOException {
@@ -1905,9 +1252,6 @@ public class HBaseAdmin implements Admin {
     compact(tableName, columnFamily, true, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
   throws IOException {
@@ -2001,19 +1345,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Move the region <code>r</code> to <code>dest</code>.
-   * @param encodedRegionName The encoded region name; i.e. the hash that makes
-   * up the region name suffix: e.g. if regionname is
-   * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
-   * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
-   * @param destServerName The servername of the destination regionserver.  If
-   * passed the empty byte array we'll assign to a random server.  A server name
-   * is made of host, port and startcode.  Here is an example:
-   * <code> host187.example.com,60020,1289493121758</code>
-   * @throws UnknownRegionException Thrown if we can't find a region named
-   * <code>encodedRegionName</code>
-   */
   @Override
   public void move(final byte [] encodedRegionName, final byte [] destServerName)
       throws IOException {
@@ -2034,13 +1365,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * @param regionName
-   *          Region name to assign.
-   * @throws MasterNotRunningException
-   * @throws ZooKeeperConnectionException
-   * @throws IOException
-   */
   @Override
   public void assign(final byte[] regionName) throws MasterNotRunningException,
       ZooKeeperConnectionException, IOException {
@@ -2056,20 +1380,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Unassign a region from current hosting regionserver.  Region will then be
-   * assigned to a regionserver chosen at random.  Region could be reassigned
-   * back to the same server.  Use {@link #move(byte[], byte[])} if you want
-   * to control the region movement.
-   * @param regionName Region to unassign. Will clear any existing RegionPlan
-   * if one found.
-   * @param force If true, force unassign (Will remove region from
-   * regions-in-transition too if present. If results in double assignment
-   * use hbck -fix to resolve. To be used by experts).
-   * @throws MasterNotRunningException
-   * @throws ZooKeeperConnectionException
-   * @throws IOException
-   */
   @Override
   public void unassign(final byte [] regionName, final boolean force)
   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
@@ -2085,18 +1395,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Offline specified region from master's in-memory state. It will not attempt to reassign the
-   * region as in unassign. This API can be used when a region not served by any region server and
-   * still online as per Master's in memory state. If this API is incorrectly used on active region
-   * then master will loose track of that region.
-   *
-   * This is a special method that should be used by experts or hbck.
-   *
-   * @param regionName
-   *          Region to offline.
-   * @throws IOException
-   */
   @Override
   public void offline(final byte [] regionName)
   throws IOException {
@@ -2109,12 +1407,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Turn the load balancer on or off.
-   * @param on If true, enable balancer. If false, disable balancer.
-   * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
-   * @return Previous balancer value
-   */
   @Override
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws IOException {
@@ -2128,12 +1420,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Invoke the balancer.  Will run the balancer and if regions to move, it will
-   * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
-   * logs.
-   * @return True if balancer ran, false otherwise.
-   */
   @Override
   public boolean balancer() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -2154,12 +1440,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Query the state of the balancer from the Master. It's not a guarantee that the balancer is
-   * actually running this very moment, but that it will run.
-   *
-   * @return True if the balancer is enabled, false otherwise.
-   */
   @Override
   public boolean isBalancerEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -2171,11 +1451,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Invoke region normalizer. Can NOT run for various reasons.  Check logs.
-   *
-   * @return True if region normalizer ran, false otherwise.
-   */
   @Override
   public boolean normalize() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -2187,11 +1462,7 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Query the current state of the region normalizer
-   *
-   * @return true if region normalizer is enabled, false otherwise.
-   */
+  @Override
   public boolean isNormalizerEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -2202,11 +1473,7 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Turn region normalizer on or off.
-   *
-   * @return Previous normalizer value
-   */
+  @Override
   public boolean setNormalizerRunning(final boolean on) throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
@@ -2218,15 +1485,8 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Enable/Disable the catalog janitor
-   * @param enable if true enables the catalog janitor
-   * @return the previous state
-   * @throws MasterNotRunningException
-   */
   @Override
-  public boolean enableCatalogJanitor(final boolean enable)
-      throws IOException {
+  public boolean enableCatalogJanitor(final boolean enable) throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
       @Override
       public Boolean call(int callTimeout) throws ServiceException {
@@ -2236,11 +1496,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Ask for a scan of the catalog table
-   * @return the number of entries cleaned
-   * @throws MasterNotRunningException
-   */
   @Override
   public int runCatalogScan() throws IOException {
     return executeCallable(new MasterCallable<Integer>(getConnection()) {
@@ -2252,10 +1507,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Query on the catalog janitor state (Enabled/Disabled?)
-   * @throws org.apache.hadoop.hbase.MasterNotRunningException
-   */
   @Override
   public boolean isCatalogJanitorEnabled() throws IOException {
     return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@@ -2319,21 +1570,13 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
-  public void split(final TableName tableName)
-    throws IOException {
+  public void split(final TableName tableName) throws IOException {
     split(tableName, null);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
-  public void splitRegion(final byte[] regionName)
-    throws IOException {
+  public void splitRegion(final byte[] regionName) throws IOException {
     splitRegion(regionName, null);
   }
 
@@ -2341,8 +1584,7 @@ public class HBaseAdmin implements Admin {
    * {@inheritDoc}
    */
   @Override
-  public void split(final TableName tableName, final byte [] splitPoint)
-  throws IOException {
+  public void split(final TableName tableName, final byte [] splitPoint) throws IOException {
     ZooKeeperWatcher zookeeper = null;
     try {
       checkTableExists(tableName);
@@ -2373,12 +1615,8 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
-  public void splitRegion(final byte[] regionName, final byte [] splitPoint)
-  throws IOException {
+  public void splitRegion(final byte[] regionName, final byte [] splitPoint) throws IOException {
     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
     if (regionServerPair == null) {
       throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
@@ -2406,20 +1644,6 @@ public class HBaseAdmin implements Admin {
     ProtobufUtil.split(admin, hri, splitPoint);
   }
 
-  /**
-   * Modify an existing table, more IRB friendly version. Asynchronous operation.
-   * This means that it may be a while before your schema change is updated across all of the
-   * table. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
-   * It may throw ExecutionException if there was an error while executing the operation
-   * or TimeoutException in case the wait timeout was not long enough to allow the
-   * operation to complete.
-   *
-   * @param tableName name of table.
-   * @param htd modified description of the table
-   * @throws IOException if a remote or network exception occurs
-   * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
-   *     operation to complete.
-   */
   @Override
   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
   throws IOException {
@@ -2468,21 +1692,11 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
-  throws IOException {
-    modifyTable(TableName.valueOf(tableName), htd);
-  }
-
-  public void modifyTable(final String tableName, final HTableDescriptor htd)
-  throws IOException {
-    modifyTable(TableName.valueOf(tableName), htd);
-  }
-
   /**
    * @param regionName Name of a region.
    * @return a pair of HRegionInfo and ServerName if <code>regionName</code> is
    *  a verified region name (we call {@link
-   *  MetaTableAccessor#getRegion(HConnection, byte[])}
+   *  MetaTableAccessor#getRegionLocation(HConnection, byte[])}
    *  else null.
    * Throw IllegalArgumentException if <code>regionName</code> is null.
    * @throws IOException
@@ -2573,10 +1787,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Shuts down the HBase cluster
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public synchronized void shutdown() throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2588,12 +1798,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Shuts down the current HBase master only.
-   * Does not shutdown the cluster.
-   * @see #shutdown()
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public synchronized void stopMaster() throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2605,12 +1809,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Stop the designated regionserver
-   * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
-   * <code>example.org:1234</code>
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public synchronized void stopRegionServer(final String hostnamePort)
   throws IOException {
@@ -2627,11 +1825,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-
-  /**
-   * @return cluster status
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public ClusterStatus getClusterStatus() throws IOException {
     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
@@ -2643,19 +1836,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * @return Configuration used by the instance.
-   */
   @Override
   public Configuration getConfiguration() {
     return this.conf;
   }
 
-  /**
-   * Create a new namespace
-   * @param descriptor descriptor which describes the new namespace
-   * @throws IOException
-   */
   @Override
   public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2671,11 +1856,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Modify an existing namespace
-   * @param descriptor descriptor which describes the new namespace
-   * @throws IOException
-   */
   @Override
   public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2688,11 +1868,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
-   * @param name namespace name
-   * @throws IOException
-   */
   @Override
   public void deleteNamespace(final String name) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -2705,12 +1880,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Get a namespace descriptor by name
-   * @param name name of namespace descriptor
-   * @return A descriptor
-   * @throws IOException
-   */
   @Override
   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
     return
@@ -2724,11 +1893,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * List available namespace descriptors
-   * @return List of descriptors
-   * @throws IOException
-   */
   @Override
   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
     return
@@ -2747,11 +1911,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * List procedures
-   * @return procedure list
-   * @throws IOException
-   */
   @Override
   public ProcedureInfo[] listProcedures() throws IOException {
     return
@@ -2769,12 +1928,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * Get list of table descriptors by namespace
-   * @param name namespace name
-   * @return A descriptor
-   * @throws IOException
-   */
   @Override
   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
     return
@@ -2794,12 +1947,6 @@ public class HBaseAdmin implements Admin {
         });
   }
 
-  /**
-   * Get list of table names by namespace
-   * @param name namespace name
-   * @return The list of table names in the namespace
-   * @throws IOException
-   */
   @Override
   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
     return
@@ -2858,13 +2005,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * get the regions of a given table.
-   *
-   * @param tableName the name of the table
-   * @return Ordered list of {@link HRegionInfo}.
-   * @throws IOException
-   */
   @Override
   public List<HRegionInfo> getTableRegions(final TableName tableName)
   throws IOException {
@@ -2884,21 +2024,10 @@ public class HBaseAdmin implements Admin {
     return regions;
   }
 
-  public List<HRegionInfo> getTableRegions(final byte[] tableName)
-  throws IOException {
-    return getTableRegions(TableName.valueOf(tableName));
-  }
-
   @Override
   public synchronized void close() throws IOException {
   }
 
-  /**
-   * Get tableDescriptors
-   * @param tableNames List of table names
-   * @return HTD[] the tableDescriptor
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
   throws IOException {
@@ -2933,12 +2062,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Get tableDescriptors
-   * @param names List of table names
-   * @return HTD[] the tableDescriptor
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public HTableDescriptor[] getTableDescriptors(List<String> names)
   throws IOException {
@@ -3015,18 +2138,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public CompactionState getCompactionState(final TableName tableName)
   throws IOException {
     return getCompactionState(tableName, CompactType.NORMAL);
   }
 
-  /**
-   * {@inheritDoc}
-   */
   @Override
   public CompactionState getCompactionStateForRegion(final byte[] regionName)
   throws IOException {
@@ -3049,22 +2166,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
-   * taken. If the table is disabled, an offline snapshot is taken.
-   * <p>
-   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-   * snapshot with the same name (even a different type or with different parameters) will fail with
-   * a {@link SnapshotCreationException} indicating the duplicate naming.
-   * <p>
-   * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
-   * @param snapshotName name of the snapshot to be created
-   * @param tableName name of the table for which snapshot is created
-   * @throws IOException if a remote or network exception occurs
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
   @Override
   public void snapshot(final String snapshotName,
                        final TableName tableName) throws IOException,
@@ -3072,86 +2173,16 @@ public class HBaseAdmin implements Admin {
     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
   }
 
-  public void snapshot(final String snapshotName,
-                       final String tableName) throws IOException,
-      SnapshotCreationException, IllegalArgumentException {
-    snapshot(snapshotName, TableName.valueOf(tableName),
-        SnapshotDescription.Type.FLUSH);
-  }
-
-  /**
-   * Create snapshot for the given table of given flush type.
-   * <p>
-   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-   * snapshot with the same name (even a different type or with different parameters) will fail with
-   * a {@link SnapshotCreationException} indicating the duplicate naming.
-   * <p>
-   * Snapshot names follow the same naming constraints as tables in HBase.
-   * @param snapshotName name of the snapshot to be created
-   * @param tableName name of the table for which snapshot is created
-   * @param flushType if the snapshot should be taken without flush memstore first
-   * @throws IOException if a remote or network exception occurs
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
-   public void snapshot(final byte[] snapshotName, final byte[] tableName,
-                       final SnapshotDescription.Type flushType) throws
-      IOException, SnapshotCreationException, IllegalArgumentException {
-      snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType);
-  }
-  /**
-   public void snapshot(final String snapshotName,
-    * Create a timestamp consistent snapshot for the given table.
-                        final byte[] tableName) throws IOException,
-    * <p>
-    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-    * snapshot with the same name (even a different type or with different parameters) will fail
-    * with a {@link SnapshotCreationException} indicating the duplicate naming.
-    * <p>
-    * Snapshot names follow the same naming constraints as tables in HBase.
-    * @param snapshotName name of the snapshot to be created
-    * @param tableName name of the table for which snapshot is created
-    * @throws IOException if a remote or network exception occurs
-    * @throws SnapshotCreationException if snapshot creation failed
-    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-    */
-  @Override
-  public void snapshot(final byte[] snapshotName,
-                       final TableName tableName) throws IOException,
-      SnapshotCreationException, IllegalArgumentException {
+  @Override
+  public void snapshot(final byte[] snapshotName, final TableName tableName)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
     snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
   }
 
-  public void snapshot(final byte[] snapshotName,
-                       final byte[] tableName) throws IOException,
-      SnapshotCreationException, IllegalArgumentException {
-    snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
-      SnapshotDescription.Type.FLUSH);
-  }
-
-  /**
-   * Create typed snapshot of the table.
-   * <p>
-   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-   * snapshot with the same name (even a different type or with different parameters) will fail with
-   * a {@link SnapshotCreationException} indicating the duplicate naming.
-   * <p>
-   * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
-   * <p>
-   * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
-   *          snapshots stored on the cluster
-   * @param tableName name of the table to snapshot
-   * @param type type of snapshot to take
-   * @throws IOException we fail to reach the master
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
   @Override
-  public void snapshot(final String snapshotName,
-                       final TableName tableName,
-                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
-      IllegalArgumentException {
+  public void snapshot(final String snapshotName, final TableName tableName,
+      SnapshotDescription.Type type)
+      throws IOException, SnapshotCreationException, IllegalArgumentException {
     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
     builder.setTable(tableName.getNameAsString());
     builder.setName(snapshotName);
@@ -3159,41 +2190,6 @@ public class HBaseAdmin implements Admin {
     snapshot(builder.build());
   }
 
-  public void snapshot(final String snapshotName,
-                       final String tableName,
-                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
-      IllegalArgumentException {
-    snapshot(snapshotName, TableName.valueOf(tableName), type);
-  }
-
-  public void snapshot(final String snapshotName,
-                       final byte[] tableName,
-                      SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
-      IllegalArgumentException {
-    snapshot(snapshotName, TableName.valueOf(tableName), type);
-  }
-
-  /**
-   * Take a snapshot and wait for the server to complete that snapshot (blocking).
-   * <p>
-   * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
-   * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
-   * time for a single cluster).
-   * <p>
-   * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
-   * snapshot with the same name (even a different type or with different parameters) will fail with
-   * a {@link SnapshotCreationException} indicating the duplicate naming.
-   * <p>
-   * Snapshot names follow the same naming constraints as tables in HBase. See
-   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
-   * <p>
-   * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
-   * unless you are sure about the type of snapshot that you want to take.
-   * @param snapshot snapshot to take
-   * @throws IOException or we lose contact with the master.
-   * @throws SnapshotCreationException if snapshot failed to be taken
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
   @Override
   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
       IllegalArgumentException {
@@ -3235,16 +2231,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
-   * <p>
-   * Only a single snapshot should be taken at a time, or results may be undefined.
-   * @param snapshot snapshot to take
-   * @return response from the server indicating the max time to wait for the snapshot
-   * @throws IOException if the snapshot did not succeed or we lose contact with the master.
-   * @throws SnapshotCreationException if snapshot creation failed
-   * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
-   */
   @Override
   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
       SnapshotCreationException {
@@ -3260,26 +2246,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Check the current state of the passed snapshot.
-   * <p>
-   * There are three possible states:
-   * <ol>
-   * <li>running - returns <tt>false</tt></li>
-   * <li>finished - returns <tt>true</tt></li>
-   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
-   * </ol>
-   * <p>
-   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
-   * run/started since the snapshot your are checking, you will recieve an
-   * {@link UnknownSnapshotException}.
-   * @param snapshot description of the snapshot to check
-   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
-   *         running
-   * @throws IOException if we have a network issue
-   * @throws HBaseSnapshotException if the snapshot failed
-   * @throws UnknownSnapshotException if the requested snapshot is unknown
-   */
   @Override
   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
@@ -3293,38 +2259,12 @@ public class HBaseAdmin implements Admin {
     }).getDone();
   }
 
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled)
-   * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
-   * is set to true, a snapshot of the current table is taken
-   * before executing the restore operation.
-   * In case of restore failure, the failsafe snapshot will be restored.
-   * If the restore completes without problem the failsafe snapshot is deleted.
-   *
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   */
   @Override
   public void restoreSnapshot(final byte[] snapshotName)
       throws IOException, RestoreSnapshotException {
     restoreSnapshot(Bytes.toString(snapshotName));
   }
 
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled)
-   * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
-   * is set to true, a snapshot of the current table is taken
-   * before executing the restore operation.
-   * In case of restore failure, the failsafe snapshot will be restored.
-   * If the restore completes without problem the failsafe snapshot is deleted.
-   *
-   * @param snapshotName name of the snapshot to restore
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   */
   @Override
   public void restoreSnapshot(final String snapshotName)
       throws IOException, RestoreSnapshotException {
@@ -3333,44 +2273,12 @@ public class HBaseAdmin implements Admin {
     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
   }
 
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled)
-   * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
-   * before executing the restore operation.
-   * In case of restore failure, the failsafe snapshot will be restored.
-   * If the restore completes without problem the failsafe snapshot is deleted.
-   *
-   * The failsafe snapshot name is configurable by using the property
-   * "hbase.snapshot.restore.failsafe.name".
-   *
-   * @param snapshotName name of the snapshot to restore
-   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   */
   @Override
   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
       throws IOException, RestoreSnapshotException {
     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
   }
 
-  /**
-   * Restore the specified snapshot on the original table. (The table must be disabled)
-   * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
-   * before executing the restore operation.
-   * In case of restore failure, the failsafe snapshot will be restored.
-   * If the restore completes without problem the failsafe snapshot is deleted.
-   *
-   * The failsafe snapshot name is configurable by using the property
-   * "hbase.snapshot.restore.failsafe.name".
-   *
-   * @param snapshotName name of the snapshot to restore
-   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
-   * @throws IOException if a remote or network exception occurs
-   * @throws RestoreSnapshotException if snapshot failed to be restored
-   * @throws IllegalArgumentException if the restore request is formatted incorrectly
-   */
   @Override
   public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
       throws IOException, RestoreSnapshotException {
@@ -3445,64 +2353,12 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Create a new table by cloning the snapshot content.
-   *
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   */
-  public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException {
-    cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
-  }
-
-  /**
-   * Create a new table by cloning the snapshot content.
-   *
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   */
   @Override
   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
     cloneSnapshot(Bytes.toString(snapshotName), tableName);
   }
 
-
-
-  /**
-   * Create a new table by cloning the snapshot content.
-   *
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   */
-  public void cloneSnapshot(final String snapshotName, final String tableName)
-      throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
-    cloneSnapshot(snapshotName, TableName.valueOf(tableName));
-  }
-
-  /**
-   * Create a new table by cloning the snapshot content.
-   *
-   * @param snapshotName name of the snapshot to be cloned
-   * @param tableName name of the table where the snapshot will be restored
-   * @throws IOException if a remote or network exception occurs
-   * @throws TableExistsException if table to be created already exists
-   * @throws RestoreSnapshotException if snapshot failed to be cloned
-   * @throws IllegalArgumentException if the specified table has not a valid name
-   */
   @Override
   public void cloneSnapshot(final String snapshotName, final TableName tableName)
       throws IOException, TableExistsException, RestoreSnapshotException {
@@ -3513,20 +2369,9 @@ public class HBaseAdmin implements Admin {
     waitUntilTableIsEnabled(tableName);
   }
 
-  /**
-   * Execute a distributed procedure on a cluster synchronously with return data
-   *
-   * @param signature A distributed procedure is uniquely identified
-   * by its signature (default the root ZK node name of the procedure).
-   * @param instance The instance name of the procedure. For some procedures, this parameter is
-   * optional.
-   * @param props Property/Value pairs of properties passing to the procedure
-   * @return data returned after procedure execution. null if no return data.
-   * @throws IOException
-   */
   @Override
-  public byte[] execProcedureWithRet(String signature, String instance,
-      Map<String, String> props) throws IOException {
+  public byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props)
+      throws IOException {
     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
     builder.setSignature(signature).setInstance(instance);
     for (Entry<String, String> entry : props.entrySet()) {
@@ -3548,19 +2393,10 @@ public class HBaseAdmin implements Admin {
 
     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
   }
-  /**
-   * Execute a distributed procedure on a cluster.
-   *
-   * @param signature A distributed procedure is uniquely identified
-   * by its signature (default the root ZK node name of the procedure).
-   * @param instance The instance name of the procedure. For some procedures, this parameter is
-   * optional.
-   * @param props Property/Value pairs of properties passing to the procedure
-   * @throws IOException
-   */
+
   @Override
-  public void execProcedure(String signature, String instance,
-      Map<String, String> props) throws IOException {
+  public void execProcedure(String signature, String instance, Map<String, String> props)
+      throws IOException {
     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
     builder.setSignature(signature).setInstance(instance);
     for (Entry<String, String> entry : props.entrySet()) {
@@ -3608,23 +2444,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Check the current state of the specified procedure.
-   * <p>
-   * There are three possible states:
-   * <ol>
-   * <li>running - returns <tt>false</tt></li>
-   * <li>finished - returns <tt>true</tt></li>
-   * <li>finished with error - throws the exception that caused the procedure to fail</li>
-   * </ol>
-   * <p>
-   *
-   * @param signature The signature that uniquely identifies a procedure
-   * @param instance The instance name of the procedure
-   * @param props Property/Value pairs of properties passing to the procedure
-   * @return true if the specified procedure is finished successfully, false if it is still running
-   * @throws IOException if the specified procedure finished with error
-   */
   @Override
   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
       throws IOException {
@@ -3656,8 +2475,7 @@ public class HBaseAdmin implements Admin {
    * @throws RestoreSnapshotException if snapshot failed to be restored
    * @throws IllegalArgumentException if the restore request is formatted incorrectly
    */
-  private void internalRestoreSnapshot(final String snapshotName, final TableName
-      tableName)
+  private void internalRestoreSnapshot(final String snapshotName, final TableName tableName)
       throws IOException, RestoreSnapshotException {
     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
@@ -3722,11 +2540,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * List completed snapshots.
-   * @return a list of snapshot descriptors for completed snapshots
-   * @throws IOException if a network error occurs
-   */
   @Override
   public List<SnapshotDescription> listSnapshots() throws IOException {
     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
@@ -3738,25 +2551,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * List all the completed snapshots matching the given regular expression.
-   *
-   * @param regex The regular expression to match against
-   * @return - returns a List of SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
     return listSnapshots(Pattern.compile(regex));
   }
 
-  /**
-   * List all the completed snapshots matching the given pattern.
-   *
-   * @param pattern The compiled regular expression to match against
-   * @return - returns a List of SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
@@ -3769,28 +2568,12 @@ public class HBaseAdmin implements Admin {
     return matched;
   }
 
-  /**
-   * List all the completed snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @return returns a List of completed SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
       String snapshotNameRegex) throws IOException {
     return listTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
   }
 
-  /**
-   * List all the completed snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
-   * @param tableNamePattern The compiled table name regular expression to match against
-   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
-   * @return returns a List of completed SnapshotDescription
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
       Pattern snapshotNamePattern) throws IOException {
@@ -3808,21 +2591,11 @@ public class HBaseAdmin implements Admin {
     return tableSnapshots;
   }
 
-  /**
-   * Delete an existing snapshot.
-   * @param snapshotName name of the snapshot
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
     deleteSnapshot(Bytes.toString(snapshotName));
   }
 
-  /**
-   * Delete an existing snapshot.
-   * @param snapshotName name of the snapshot
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteSnapshot(final String snapshotName) throws IOException {
     // make sure the snapshot is possibly valid
@@ -3840,21 +2613,11 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Delete existing snapshots whose names match the pattern passed.
-   * @param regex The regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteSnapshots(final String regex) throws IOException {
     deleteSnapshots(Pattern.compile(regex));
   }
 
-  /**
-   * Delete existing snapshots whose names match the pattern passed.
-   * @param pattern pattern for names of the snapshot to match
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteSnapshots(final Pattern pattern) throws IOException {
     List<SnapshotDescription> snapshots = listSnapshots(pattern);
@@ -3880,26 +2643,12 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Delete all existing snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
-   * @param tableNameRegex The table name regular expression to match against
-   * @param snapshotNameRegex The snapshot name regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex)
       throws IOException {
     deleteTableSnapshots(Pattern.compile(tableNameRegex), Pattern.compile(snapshotNameRegex));
   }
 
-  /**
-   * Delete all existing snapshots matching the given table name regular expression and snapshot
-   * name regular expression.
-   * @param tableNamePattern The compiled table name regular expression to match against
-   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
       throws IOException {
@@ -3914,12 +2663,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Apply the new quota settings.
-   *
-   * @param quota the quota settings
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public void setQuota(final QuotaSettings quota) throws IOException {
     executeCallable(new MasterCallable<Void>(getConnection()) {
@@ -3931,13 +2674,6 @@ public class HBaseAdmin implements Admin {
     });
   }
 
-  /**
-   * Return a Quota Scanner to list the quotas based on the filter.
-   *
-   * @param filter the quota settings filter
-   * @return the quota scanner
-   * @throws IOException if a remote or network exception occurs
-   */
   @Override
   public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
     return QuotaRetriever.open(conf, filter);
@@ -3958,27 +2694,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
-   * connected to the active master.
-   *
-   * <p>
-   * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
-   * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
-   * </p>
-   *
-   * <div style="background-color: #cccccc; padding: 2px">
-   * <blockquote><pre>
-   * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
-   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
-   * MyCallRequest request = MyCallRequest.newBuilder()
-   *     ...
-   *     .build();
-   * MyCallResponse response = service.myCall(null, request);
-   * </pre></blockquote></div>
-   *
-   * @return A MasterCoprocessorRpcChannel instance
-   */
   @Override
   public CoprocessorRpcChannel coprocessorService() {
     return new MasterCoprocessorRpcChannel(connection);
@@ -4000,28 +2715,6 @@ public class HBaseAdmin implements Admin {
     }
   }
 
-  /**
-   * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
-   * connected to the passed region server.
-   *
-   * <p>
-   * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
-   * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
-   * </p>
-   *
-   * <div style="background-color: #cccccc; padding: 2px">
-   * <blockquote><pre>
-   * CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
-   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
-   * MyCallRequest request = MyCallRequest.newBuilder()
-   *     ...
-   *     .build();
-   * MyCallResponse response = service.myCall(null, request);
-   * </pre></blockquote></div>
-   *
-   * @param sn the server name to which the endpoint call is made
-   * @return A RegionServerCoprocessorRpcChannel instance
-   */
   @Override
   public CoprocessorRpcChannel coprocessorService(ServerName sn) {
     return new RegionServerCoprocessorRpcChannel(connection, sn);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bebcc09f/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index dffde55..d9c7caa 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -718,7 +718,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
         }
         TableName tableName = selected.getTableName();
         LOG.info("Deleting column family: " + cfd + " from table: " + tableName);
-        admin.deleteColumn(tableName, cfd.getName());
+        admin.deleteColumnFamily(tableName, cfd.getName());
         // assertion
         HTableDescriptor freshTableDesc = admin.getTableDescriptor(tableName);
         Assert.assertFalse("Column family: " + cfd + " was not added",