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 2016/02/09 00:13:41 UTC

[01/32] hbase git commit: HBASE-15200 ZooKeeper znode ACL checks should only compare the shortname

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 2f5767376 -> 3aff98c75


HBASE-15200 ZooKeeper znode ACL checks should only compare the shortname


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

Branch: refs/heads/hbase-12439
Commit: 6256ce4e63bbff16f0678a1fbd6c33649c373f29
Parents: 2f57673
Author: Andrew Purtell <ap...@apache.org>
Authored: Mon Feb 1 09:48:16 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 3 10:43:20 2016 -0800

----------------------------------------------------------------------
 .../hbase/zookeeper/ZooKeeperWatcher.java       | 56 ++++++++++++++++++--
 1 file changed, 52 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6256ce4e/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index f7a2175..0bc75eb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -26,6 +26,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -126,6 +128,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
 
   private final Exception constructorCaller;
 
+  /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */
+  private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
   /**
    * Instantiate a ZooKeeper connection and watcher.
    * @param identifier string that is passed to RecoverableZookeeper to be used as
@@ -215,6 +220,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
    */
   public void checkAndSetZNodeAcls() {
     if (!ZKUtil.isSecureZooKeeper(getConfiguration())) {
+      LOG.info("not a secure deployment, proceeding");
       return;
     }
 
@@ -259,6 +265,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
    * @throws IOException
    */
   private boolean isBaseZnodeAclSetup(List<ACL> acls) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Checking znode ACLs");
+    }
     String[] superUsers = conf.getStrings(Superusers.SUPERUSER_CONF_KEY);
     // Check whether ACL set for all superusers
     if (superUsers != null && !checkACLForSuperUsers(superUsers, acls)) {
@@ -270,6 +279,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     String hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName();
 
     if (acls.isEmpty()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ACL is empty");
+      }
       return false;
     }
 
@@ -280,17 +292,45 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
       // and one for the hbase user
       if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
         if (perms != Perms.READ) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
+              id, perms, Perms.READ));
+          }
           return false;
         }
       } else if (superUsers != null && isSuperUserId(superUsers, id)) {
         if (perms != Perms.ALL) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
+              id, perms, Perms.ALL));
+          }
           return false;
         }
-      } else if (new Id("sasl", hbaseUser).equals(id)) {
-        if (perms != Perms.ALL) {
+      } else if ("sasl".equals(id.getScheme())) {
+        String name = id.getId();
+        // If ZooKeeper recorded the Kerberos full name in the ACL, use only the shortname
+        Matcher match = NAME_PATTERN.matcher(name);
+        if (match.matches()) {
+          name = match.group(1);
+        }
+        if (name.equals(hbaseUser)) {
+          if (perms != Perms.ALL) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
+                id, perms, Perms.ALL));
+            }
+            return false;
+          }
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Unexpected shortname in SASL ACL: " + id);
+          }
           return false;
         }
       } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("unexpected ACL id '" + id + "'");
+        }
         return false;
       }
     }
@@ -306,8 +346,16 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
       // TODO: Validate super group members also when ZK supports setting node ACL for groups.
       if (!user.startsWith(AuthUtil.GROUP_PREFIX)) {
         for (ACL acl : acls) {
-          if (user.equals(acl.getId().getId()) && acl.getPerms() == Perms.ALL) {
-            hasAccess = true;
+          if (user.equals(acl.getId().getId())) {
+            if (acl.getPerms() == Perms.ALL) {
+              hasAccess = true;
+            } else {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug(String.format(
+                  "superuser '%s' does not have correct permissions: have %0x, want %0x",
+                  acl.getId().getId(), acl.getPerms(), Perms.ALL));
+              }
+            }
             break;
           }
         }


[07/32] hbase git commit: HBASE-15120 Undo aggressive load balancer logging at tens of lines per millisecond

Posted by sy...@apache.org.
HBASE-15120 Undo aggressive load balancer logging at tens of lines per millisecond


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

Branch: refs/heads/hbase-12439
Commit: c177cfed416e0cf72ae6131c5e98c99672bef3cc
Parents: a69272e
Author: stack <st...@apache.org>
Authored: Thu Feb 4 14:24:05 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 4 14:24:05 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/balancer/BaseLoadBalancer.java       | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c177cfed/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 20f4169..8680c89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -816,9 +816,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           i++;
           lowestLocalityServerIndex = serverIndicesSortedByLocality[i];
         }
-        LOG.debug("Lowest locality region server with non zero regions is "
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Lowest locality region server with non zero regions is "
             + servers[lowestLocalityServerIndex].getHostname() + " with locality "
             + localityPerServer[lowestLocalityServerIndex]);
+        }
         return lowestLocalityServerIndex;
       }
     }
@@ -841,9 +843,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
             lowestLocalityRegionIndex = j;
           }
         }
-        LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
+        if (LOG.isTraceEnabled()) {
+          LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
             + " and its region server contains " + regionsPerServer[serverIndex].length
             + " regions");
+        }
         return regionsPerServer[serverIndex][lowestLocalityRegionIndex];
       } else {
         return -1;


[02/32] hbase git commit: HBASE-HBASE-15203 Reduce garbage created by path.toString() during Checksum verification (Ram)

Posted by sy...@apache.org.
HBASE-HBASE-15203 Reduce garbage created by path.toString() during
Checksum verification (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 2cf8af5bf1d501156cbb3b421cf75c1051ead7d9
Parents: 6256ce4
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Feb 4 11:44:46 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Feb 4 11:44:46 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |  8 +++-----
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 21 ++++++++++----------
 2 files changed, 14 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2cf8af5b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
index 61862eb..402caa8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -87,7 +86,7 @@ public class ChecksumUtil {
    * The header is extracted from the specified HFileBlock while the
    * data-to-be-verified is extracted from 'data'.
    */
-  static boolean validateBlockChecksum(Path path, HFileBlock block,
+  static boolean validateBlockChecksum(String pathName, HFileBlock block,
     byte[] data, int hdrSize) throws IOException {
 
     // If this is an older version of the block that does not have
@@ -120,14 +119,13 @@ public class ChecksumUtil {
       LOG.info("length of data = " + data.length
           + " OnDiskDataSizeWithHeader = " + sizeWithHeader
           + " checksum type = " + cktype.getName()
-          + " file =" + path.toString()
+          + " file =" + pathName
           + " header size = " + hdrSize
           + " bytesPerChecksum = " + bytesPerChecksum);
     }
     try {
       dataChecksum.verifyChunkedSums(ByteBuffer.wrap(data, 0, sizeWithHeader),
-          ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader),
-                          path.toString(), 0);
+          ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader), pathName, 0);
     } catch (ChecksumException e) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2cf8af5b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 0a25825..e0719aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1353,21 +1353,22 @@ public class HFileBlock implements Cacheable {
     /** The filesystem used to access data */
     protected HFileSystem hfs;
 
-    /** The path (if any) where this data is coming from */
-    protected Path path;
-
     private final Lock streamLock = new ReentrantLock();
 
     /** The default buffer size for our buffered streams */
     public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
 
     protected HFileContext fileContext;
+    // Cache the fileName
+    protected String pathName;
 
     public FSReaderImpl(FSDataInputStreamWrapper stream, long fileSize, HFileSystem hfs, Path path,
         HFileContext fileContext) throws IOException {
       this.fileSize = fileSize;
       this.hfs = hfs;
-      this.path = path;
+      if (path != null) {
+        this.pathName = path.toString();
+      }
       this.fileContext = fileContext;
       this.hdrSize = headerSize(fileContext.isUseHBaseChecksum());
 
@@ -1508,13 +1509,13 @@ public class HFileBlock implements Cacheable {
                          doVerificationThruHBaseChecksum);
       if (blk == null) {
         HFile.LOG.warn("HBase checksum verification failed for file " +
-                       path + " at offset " +
+                       pathName + " at offset " +
                        offset + " filesize " + fileSize +
                        ". Retrying read with HDFS checksums turned on...");
 
         if (!doVerificationThruHBaseChecksum) {
           String msg = "HBase checksum verification failed for file " +
-                       path + " at offset " +
+                       pathName + " at offset " +
                        offset + " filesize " + fileSize +
                        " but this cannot happen because doVerify is " +
                        doVerificationThruHBaseChecksum;
@@ -1536,13 +1537,13 @@ public class HFileBlock implements Cacheable {
                                     doVerificationThruHBaseChecksum);
         if (blk != null) {
           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
-                         path + " at offset " +
+                         pathName + " at offset " +
                          offset + " filesize " + fileSize);
         }
       }
       if (blk == null && !doVerificationThruHBaseChecksum) {
         String msg = "readBlockData failed, possibly due to " +
-                     "checksum verification failed for file " + path +
+                     "checksum verification failed for file " + pathName +
                      " at offset " + offset + " filesize " + fileSize;
         HFile.LOG.warn(msg);
         throw new IOException(msg);
@@ -1744,7 +1745,7 @@ public class HFileBlock implements Cacheable {
      */
     protected boolean validateBlockChecksum(HFileBlock block,  byte[] data, int hdrSize)
         throws IOException {
-      return ChecksumUtil.validateBlockChecksum(path, block, data, hdrSize);
+      return ChecksumUtil.validateBlockChecksum(pathName, block, data, hdrSize);
     }
 
     @Override
@@ -1754,7 +1755,7 @@ public class HFileBlock implements Cacheable {
 
     @Override
     public String toString() {
-      return "hfs=" + hfs + ", path=" + path + ", fileContext=" + fileContext;
+      return "hfs=" + hfs + ", path=" + pathName + ", fileContext=" + fileContext;
     }
   }
 


[17/32] hbase git commit: HBASE-15218 On RS crash and replay of WAL, loosing all Tags in Cells (Anoop Sam John)

Posted by sy...@apache.org.
HBASE-15218 On RS crash and replay of WAL, loosing all Tags in Cells (Anoop Sam John)


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

Branch: refs/heads/hbase-12439
Commit: 779bdf19183ed182d34d8be5b0cc92c6ba93d70b
Parents: 9c83210
Author: stack <st...@apache.org>
Authored: Fri Feb 5 10:08:20 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 10:08:20 2016 -0800

----------------------------------------------------------------------
 .../regionserver/wal/SecureWALCellCodec.java    |  6 +++---
 .../hbase/regionserver/wal/WALCellCodec.java    |  4 ++--
 ...ibilityLabelsWithDefaultVisLabelService.java | 22 ++++++++++++++++++++
 3 files changed, 27 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/779bdf19/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java
index 69181e5..603496f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureWALCellCodec.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.io.crypto.Decryptor;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.crypto.Encryptor;
@@ -60,7 +60,7 @@ public class SecureWALCellCodec extends WALCellCodec {
     this.decryptor = decryptor;
   }
 
-  static class EncryptedKvDecoder extends KeyValueCodec.KeyValueDecoder {
+  static class EncryptedKvDecoder extends KeyValueCodecWithTags.KeyValueDecoder {
 
     private Decryptor decryptor;
     private byte[] iv;
@@ -142,7 +142,7 @@ public class SecureWALCellCodec extends WALCellCodec {
 
   }
 
-  static class EncryptedKvEncoder extends KeyValueCodec.KeyValueEncoder {
+  static class EncryptedKvEncoder extends KeyValueCodecWithTags.KeyValueEncoder {
 
     private Encryptor encryptor;
     private final ThreadLocal<byte[]> iv = new ThreadLocal<byte[]>() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/779bdf19/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
index 26e0e04..05929fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.BaseDecoder;
 import org.apache.hadoop.hbase.codec.BaseEncoder;
 import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.io.util.Dictionary;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -344,7 +344,7 @@ public class WALCellCodec implements Codec {
   @Override
   public Decoder getDecoder(InputStream is) {
     return (compression == null)
-        ? new KeyValueCodec.KeyValueDecoder(is) : new CompressedKvDecoder(is, compression);
+        ? new KeyValueCodecWithTags.KeyValueDecoder(is) : new CompressedKvDecoder(is, compression);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/779bdf19/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
index 7797493..a229bdb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
+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.Result;
@@ -219,4 +220,25 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     };
     SUPERUSER.runAs(action);
   }
+
+  @Test(timeout = 60 * 1000)
+  public void testVisibilityLabelsOnWALReplay() throws Exception {
+    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    try (Table table = createTableAndWriteDataWithLabels(tableName,
+        "(" + SECRET + "|" + CONFIDENTIAL + ")", PRIVATE);) {
+      List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
+          .getRegionServerThreads();
+      for (RegionServerThread rsThread : regionServerThreads) {
+        rsThread.getRegionServer().abort("Aborting ");
+      }
+      // Start one new RS
+      RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer();
+      waitForLabelsRegionAvailability(rs.getRegionServer());
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+    }
+  }
 }


[28/32] hbase git commit: HBASE-15221 Reload the cache on re-tried puts in HTableMultiplexer and adds a close() method to HTableMultiplexer

Posted by sy...@apache.org.
HBASE-15221 Reload the cache on re-tried puts in HTableMultiplexer and adds a close() method to HTableMultiplexer

When a Put fails due to a NotServingRegionException, the cached location
of that Region is never cleared. Thus, subsequent calls to resubmit
the Put will fail in the same way as the original, never determining
the new location of the Region.

If the Connection is not closed by the user before the Multiplexer
is discarded, it will leak resources and could cause resource
issues.

Signed-off-by: Sean Busbey <bu...@cloudera.com>


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

Branch: refs/heads/hbase-12439
Commit: dfd8a31a130ffdad382a5a6923035b1142ccdb0c
Parents: eacf7bc
Author: Josh Elser <el...@apache.org>
Authored: Sun Feb 7 23:57:23 2016 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Mon Feb 8 00:04:28 2016 -0600

----------------------------------------------------------------------
 hbase-client/pom.xml                            |   5 +
 .../hadoop/hbase/client/HTableMultiplexer.java  | 117 +++++++++--
 .../client/TestHTableMultiplexerViaMocks.java   | 193 +++++++++++++++++++
 3 files changed, 295 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dfd8a31a/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index ed20a68..e74e0d5 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -193,6 +193,11 @@
       <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/dfd8a31a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index 0349321..13e9b85 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -19,6 +19,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import java.io.IOException;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
@@ -50,8 +53,6 @@ import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
  * Each put will be sharded into different buffer queues based on its destination region server.
@@ -97,7 +98,18 @@ public class HTableMultiplexer {
    */
   public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
       throws IOException {
-    this.conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize);
+  }
+
+  /**
+   * @param conn The HBase connection.
+   * @param conf The HBase configuration
+   * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
+   *          each region server before dropping the request.
+   */
+  public HTableMultiplexer(Connection conn, Configuration conf,
+      int perRegionServerBufferQueueSize) {
+    this.conn = (ClusterConnection) conn;
     this.pool = HTable.getDefaultExecutor(conf);
     // how many times we could try in total, one more than retry number
     this.maxAttempts = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@@ -117,6 +129,18 @@ public class HTableMultiplexer {
   }
 
   /**
+   * Closes the internal {@link Connection}. Does nothing if the {@link Connection} has already
+   * been closed.
+   * @throws IOException If there is an error closing the connection.
+   */
+  @SuppressWarnings("deprecation")
+  public synchronized void close() throws IOException {
+    if (!getConnection().isClosed()) {
+      getConnection().close();
+    }
+  }
+
+  /**
    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
    * is already full.
    * @param tableName
@@ -170,13 +194,28 @@ public class HTableMultiplexer {
    * @return true if the request can be accepted by its corresponding buffer queue.
    */
   public boolean put(final TableName tableName, final Put put, int maxAttempts) {
+    return _put(tableName, put, maxAttempts, false);
+  }
+
+  /**
+   * Internal "put" which exposes a boolean flag to control whether or not the region location
+   * cache should be reloaded when trying to queue the {@link Put}.
+   * @param tableName Destination table for the Put
+   * @param put The Put to send
+   * @param maxAttempts Number of attempts to retry the {@code put}
+   * @param reloadCache Should the region location cache be reloaded
+   * @return true if the request was accepted in the queue, otherwise false
+   */
+  boolean _put(final TableName tableName, final Put put, int maxAttempts, boolean reloadCache) {
     if (maxAttempts <= 0) {
       return false;
     }
 
     try {
       HTable.validatePut(put, maxKeyValueSize);
-      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
+      // Allow mocking to get at the connection, but don't expose the connection to users.
+      ClusterConnection conn = (ClusterConnection) getConnection();
+      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), reloadCache);
       if (loc != null) {
         // Add the put pair into its corresponding queue.
         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
@@ -215,7 +254,8 @@ public class HTableMultiplexer {
     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
   }
 
-  private LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
+  @VisibleForTesting
+  LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
     FlushWorker worker = serverToFlushWorkerMap.get(addr);
     if (worker == null) {
       synchronized (this.serverToFlushWorkerMap) {
@@ -232,6 +272,11 @@ public class HTableMultiplexer {
     return worker.getQueue();
   }
 
+  @VisibleForTesting
+  ClusterConnection getConnection() {
+    return this.conn;
+  }
+
   /**
    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
    * report the number of buffered requests and the number of the failed (dropped) requests
@@ -340,10 +385,11 @@ public class HTableMultiplexer {
     }
   }
 
-  private static class PutStatus {
-    private final HRegionInfo regionInfo;
-    private final Put put;
-    private final int maxAttempCount;
+  @VisibleForTesting
+  static class PutStatus {
+    final HRegionInfo regionInfo;
+    final Put put;
+    final int maxAttempCount;
 
     public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
       this.regionInfo = regionInfo;
@@ -392,7 +438,8 @@ public class HTableMultiplexer {
     }
   }
 
-  private static class FlushWorker implements Runnable {
+  @VisibleForTesting
+  static class FlushWorker implements Runnable {
     private final HRegionLocation addr;
     private final LinkedBlockingQueue<PutStatus> queue;
     private final HTableMultiplexer multiplexer;
@@ -440,7 +487,7 @@ public class HTableMultiplexer {
       return this.maxLatency.getAndSet(0);
     }
 
-    private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
+    boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
       // Decrease the retry count
       final int retryCount = ps.maxAttempCount - 1;
 
@@ -449,10 +496,10 @@ public class HTableMultiplexer {
         return false;
       }
 
-      int cnt = retryInQueue.incrementAndGet();
-      if (cnt > maxRetryInQueue) {
+      int cnt = getRetryInQueue().incrementAndGet();
+      if (cnt > getMaxRetryInQueue()) {
         // Too many Puts in queue for resubmit, give up this
-        retryInQueue.decrementAndGet();
+        getRetryInQueue().decrementAndGet();
         return false;
       }
 
@@ -460,22 +507,21 @@ public class HTableMultiplexer {
       // The currentPut is failed. So get the table name for the currentPut.
       final TableName tableName = ps.regionInfo.getTable();
 
-      long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
-        multiplexer.maxAttempts - retryCount - 1);
+      long delayMs = getNextDelay(retryCount);
       if (LOG.isDebugEnabled()) {
         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
       }
 
-      executor.schedule(new Runnable() {
+      getExecutor().schedule(new Runnable() {
         @Override
         public void run() {
           boolean succ = false;
           try {
-            succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
+            succ = FlushWorker.this.getMultiplexer()._put(tableName, failedPut, retryCount, true);
           } finally {
-            FlushWorker.this.retryInQueue.decrementAndGet();
+            FlushWorker.this.getRetryInQueue().decrementAndGet();
             if (!succ) {
-              FlushWorker.this.totalFailedPutCount.incrementAndGet();
+              FlushWorker.this.getTotalFailedPutCount().incrementAndGet();
             }
           }
         }
@@ -483,6 +529,37 @@ public class HTableMultiplexer {
       return true;
     }
 
+    @VisibleForTesting
+    long getNextDelay(int retryCount) {
+      return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
+          multiplexer.maxAttempts - retryCount - 1);
+    }
+
+    @VisibleForTesting
+    AtomicInteger getRetryInQueue() {
+      return this.retryInQueue;
+    }
+
+    @VisibleForTesting
+    int getMaxRetryInQueue() {
+      return this.maxRetryInQueue;
+    }
+
+    @VisibleForTesting
+    AtomicLong getTotalFailedPutCount() {
+      return this.totalFailedPutCount;
+    }
+
+    @VisibleForTesting
+    HTableMultiplexer getMultiplexer() {
+      return this.multiplexer;
+    }
+
+    @VisibleForTesting
+    ScheduledExecutorService getExecutor() {
+      return this.executor;
+    }
+
     @Override
     public void run() {
       int failedCount = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/dfd8a31a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
new file mode 100644
index 0000000..38ddeb9
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.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.client;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTableMultiplexer.FlushWorker;
+import org.apache.hadoop.hbase.client.HTableMultiplexer.PutStatus;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@Category(SmallTests.class)
+public class TestHTableMultiplexerViaMocks {
+
+  private static final int NUM_RETRIES = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
+  private HTableMultiplexer mockMultiplexer;
+  private ClusterConnection mockConnection;
+  private HRegionLocation mockRegionLocation;
+  private HRegionInfo mockRegionInfo;
+
+  private TableName tableName;
+  private Put put;
+
+  @Before
+  public void setupTest() {
+    mockMultiplexer = mock(HTableMultiplexer.class);
+    mockConnection = mock(ClusterConnection.class);
+    mockRegionLocation = mock(HRegionLocation.class);
+    mockRegionInfo = mock(HRegionInfo.class);
+
+    tableName = TableName.valueOf("my_table");
+    put = new Put(getBytes("row1"));
+    put.addColumn(getBytes("f1"), getBytes("q1"), getBytes("v11"));
+    put.addColumn(getBytes("f1"), getBytes("q2"), getBytes("v12"));
+    put.addColumn(getBytes("f2"), getBytes("q1"), getBytes("v21"));
+
+    // Call the real put(TableName, Put, int) method
+    when(mockMultiplexer.put(any(TableName.class), any(Put.class), anyInt())).thenCallRealMethod();
+
+    // Return the mocked ClusterConnection
+    when(mockMultiplexer.getConnection()).thenReturn(mockConnection);
+
+    // Return the regionInfo from the region location
+    when(mockRegionLocation.getRegionInfo()).thenReturn(mockRegionInfo);
+
+    // Make sure this RegionInfo points to our table
+    when(mockRegionInfo.getTable()).thenReturn(tableName);
+  }
+
+  @Test public void useCacheOnInitialPut() throws Exception {
+    mockMultiplexer.put(tableName, put, NUM_RETRIES);
+
+    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES, false);
+  }
+
+  @Test public void nonNullLocationQueuesPut() throws Exception {
+    final LinkedBlockingQueue<PutStatus> queue = new LinkedBlockingQueue<>();
+
+    // Call the real method for _put(TableName, Put, int, boolean)
+    when(mockMultiplexer._put(any(TableName.class), any(Put.class), anyInt(), anyBoolean())).thenCallRealMethod();
+
+    // Return a region location
+    when(mockConnection.getRegionLocation(tableName, put.getRow(), false)).thenReturn(mockRegionLocation);
+    when(mockMultiplexer.getQueue(mockRegionLocation)).thenReturn(queue);
+
+    assertTrue("Put should have been queued", mockMultiplexer.put(tableName, put, NUM_RETRIES));
+
+    assertEquals(1, queue.size());
+    final PutStatus ps = queue.take();
+    assertEquals(put, ps.put);
+    assertEquals(mockRegionInfo, ps.regionInfo);
+  }
+
+  @Test public void ignoreCacheOnRetriedPut() throws Exception {
+    FlushWorker mockFlushWorker = mock(FlushWorker.class);
+    ScheduledExecutorService mockExecutor = mock(ScheduledExecutorService.class);
+    final AtomicInteger retryInQueue = new AtomicInteger(0);
+    final AtomicLong totalFailedPuts = new AtomicLong(0L);
+    final int maxRetryInQueue = 20;
+    final long delay = 100L;
+
+    final PutStatus ps = new PutStatus(mockRegionInfo, put, NUM_RETRIES);
+
+    // Call the real resubmitFailedPut(PutStatus, HRegionLocation) method
+    when(mockFlushWorker.resubmitFailedPut(any(PutStatus.class), any(HRegionLocation.class))).thenCallRealMethod();
+    // Succeed on the re-submit without caching
+    when(mockMultiplexer._put(tableName, put, NUM_RETRIES - 1, true)).thenReturn(true);
+
+    // Stub out the getters for resubmitFailedPut(PutStatus, HRegionLocation)
+    when(mockFlushWorker.getExecutor()).thenReturn(mockExecutor);
+    when(mockFlushWorker.getNextDelay(anyInt())).thenReturn(delay);
+    when(mockFlushWorker.getMultiplexer()).thenReturn(mockMultiplexer);
+    when(mockFlushWorker.getRetryInQueue()).thenReturn(retryInQueue);
+    when(mockFlushWorker.getMaxRetryInQueue()).thenReturn(maxRetryInQueue);
+    when(mockFlushWorker.getTotalFailedPutCount()).thenReturn(totalFailedPuts);
+
+    // When a Runnable is scheduled, run that Runnable
+    when(mockExecutor.schedule(any(Runnable.class), eq(delay), eq(TimeUnit.MILLISECONDS))).thenAnswer(
+        new Answer<Void>() {
+          @Override
+          public Void answer(InvocationOnMock invocation) throws Throwable {
+            // Before we run this, should have one retry in progress.
+            assertEquals(1L, retryInQueue.get());
+
+            Object[] args = invocation.getArguments();
+            assertEquals(3, args.length);
+            assertTrue("Argument should be an instance of Runnable", args[0] instanceof Runnable);
+            Runnable runnable = (Runnable) args[0];
+            runnable.run();
+            return null;
+          }
+        });
+
+    // The put should be rescheduled
+    assertTrue("Put should have been rescheduled", mockFlushWorker.resubmitFailedPut(ps, mockRegionLocation));
+
+    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES - 1, true);
+    assertEquals(0L, totalFailedPuts.get());
+    // Net result should be zero (added one before rerunning, subtracted one after running).
+    assertEquals(0L, retryInQueue.get());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test public void testConnectionClosing() throws IOException {
+    doCallRealMethod().when(mockMultiplexer).close();
+    // If the connection is not closed
+    when(mockConnection.isClosed()).thenReturn(false);
+
+    mockMultiplexer.close();
+
+    // We should close it
+    verify(mockConnection).close();
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test public void testClosingAlreadyClosedConnection() throws IOException {
+    doCallRealMethod().when(mockMultiplexer).close();
+    // If the connection is already closed
+    when(mockConnection.isClosed()).thenReturn(true);
+
+    mockMultiplexer.close();
+
+    // We should not close it again
+    verify(mockConnection, times(0)).close();
+  }
+
+  /**
+   * @return UTF-8 byte representation for {@code str}
+   */
+  private static byte[] getBytes(String str) {
+    return str.getBytes(UTF_8);
+  }
+}


[22/32] hbase git commit: HBASE-15220 Change two logs in SimpleRegionNormalizer to INFO level

Posted by sy...@apache.org.
HBASE-15220 Change two logs in SimpleRegionNormalizer to INFO level


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

Branch: refs/heads/hbase-12439
Commit: 2ce31f8945635259b4e53ac9a1d381b4df57a0fe
Parents: 31f9f2f
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 5 20:52:17 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 5 20:52:17 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2ce31f89/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index bd522b4..a99f8dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -144,7 +144,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
       // if the region is > 2 times larger than average, we split it, split
       // is more high priority normalization action than merge.
       if (regionSize > 2 * avgRegionSize) {
-        LOG.debug("Table " + table + ", large region " + hri.getRegionNameAsString() + " has size "
+        LOG.info("Table " + table + ", large region " + hri.getRegionNameAsString() + " has size "
             + regionSize + ", more than twice avg size, splitting");
         plans.add(new SplitNormalizationPlan(hri, null));
       } else {
@@ -154,7 +154,7 @@ public class SimpleRegionNormalizer implements RegionNormalizer {
         HRegionInfo hri2 = tableRegions.get(candidateIdx+1);
         long regionSize2 = getRegionSize(hri2);
         if (regionSize + regionSize2 < avgRegionSize) {
-          LOG.debug("Table " + table + ", small region size: " + regionSize
+          LOG.info("Table " + table + ", small region size: " + regionSize
             + " plus its neighbor size: " + regionSize2
             + ", less than the avg size " + avgRegionSize + ", merging them");
           plans.add(new MergeNormalizationPlan(hri, hri2));


[31/32] hbase git commit: HBASE-15158 Change order in which we do write pipeline operations; do all under row locks

Posted by sy...@apache.org.
HBASE-15158 Change order in which we do write pipeline operations; do all under row locks


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

Branch: refs/heads/hbase-12439
Commit: ec92a8a705dfec076a93454e1042645d466758f0
Parents: dfd8a31
Author: stack <st...@apache.org>
Authored: Sun Feb 7 22:56:40 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sun Feb 7 22:56:40 2016 -0800

----------------------------------------------------------------------
 .../hbase/regionserver/DefaultMemStore.java     |    7 +-
 .../hadoop/hbase/regionserver/HRegion.java      | 2216 +++++++-----------
 .../hbase/regionserver/wal/FSWALEntry.java      |    5 +-
 .../hadoop/hbase/regionserver/wal/HLogKey.java  |    4 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |   96 +-
 .../org/apache/hadoop/hbase/wal/WALKey.java     |  185 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |   12 +-
 7 files changed, 1063 insertions(+), 1462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 2984754..f61d871 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -457,7 +457,6 @@ public class DefaultMemStore implements MemStore {
    * This is called under row lock, so Get operations will still see updates
    * atomically.  Scans will only see each KeyValue update as atomic.
    *
-   * @param cells
    * @param readpoint readpoint below which we can safely remove duplicate KVs
    * @return change in memstore size
    */
@@ -578,7 +577,7 @@ public class DefaultMemStore implements MemStore {
     // last iterated Cells for cellSet and snapshot (to restore iterator state after reseek)
     private Cell cellSetItRow = null;
     private Cell snapshotItRow = null;
-    
+
     // iterator based scanning.
     private Iterator<Cell> cellSetIt;
     private Iterator<Cell> snapshotIt;
@@ -593,7 +592,7 @@ public class DefaultMemStore implements MemStore {
     // The allocator and snapshot allocator at the time of creating this scanner
     volatile MemStoreLAB allocatorAtCreation;
     volatile MemStoreLAB snapshotAllocatorAtCreation;
-    
+
     // A flag represents whether could stop skipping Cells for MVCC
     // if have encountered the next row. Only used for reversed scan
     private boolean stopSkippingCellsIfNextRow = false;
@@ -806,7 +805,7 @@ public class DefaultMemStore implements MemStore {
 
       this.cellSetIt = null;
       this.snapshotIt = null;
-      
+
       if (allocatorAtCreation != null) {
         this.allocatorAtCreation.decScannerCount();
         this.allocatorAtCreation = null;


[19/32] hbase git commit: HBASE-15214 Valid mutate Ops fail with RPC Codec in use and region moves across.

Posted by sy...@apache.org.
HBASE-15214 Valid mutate Ops fail with RPC Codec in use and region moves across.


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

Branch: refs/heads/hbase-12439
Commit: 7239056c78cc6eb2867c8865ab45821d3e51328a
Parents: 4265bf2
Author: anoopsjohn <an...@gmail.com>
Authored: Sat Feb 6 02:40:49 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Sat Feb 6 02:40:49 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     | 18 ++++-------
 .../hbase/regionserver/RSRpcServices.java       | 34 ++++++++++++++++++++
 .../hadoop/hbase/client/TestMultiParallel.java  |  4 +++
 3 files changed, 45 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7239056c/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index fe76780..e9a1223 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -543,7 +543,7 @@ public final class ProtobufUtil {
     MutationType type = proto.getMutateType();
     assert type == MutationType.PUT: type.name();
     long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
-    Put put = null;
+    Put put = proto.hasRow() ? new Put(proto.getRow().toByteArray(), timestamp) : null;
     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
     if (cellCount > 0) {
       // The proto has metadata only and the data is separate to be found in the cellScanner.
@@ -563,9 +563,7 @@ public final class ProtobufUtil {
         put.add(cell);
       }
     } else {
-      if (proto.hasRow()) {
-        put = new Put(proto.getRow().asReadOnlyByteBuffer(), timestamp);
-      } else {
+      if (put == null) {
         throw new IllegalArgumentException("row cannot be null");
       }
       // The proto has the metadata and the data itself
@@ -639,12 +637,8 @@ public final class ProtobufUtil {
   throws IOException {
     MutationType type = proto.getMutateType();
     assert type == MutationType.DELETE : type.name();
-    byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
-    long timestamp = HConstants.LATEST_TIMESTAMP;
-    if (proto.hasTimestamp()) {
-      timestamp = proto.getTimestamp();
-    }
-    Delete delete = null;
+    long timestamp = proto.hasTimestamp() ? proto.getTimestamp() : HConstants.LATEST_TIMESTAMP;
+    Delete delete = proto.hasRow() ? new Delete(proto.getRow().toByteArray(), timestamp) : null;
     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
     if (cellCount > 0) {
       // The proto has metadata only and the data is separate to be found in the cellScanner.
@@ -667,7 +661,9 @@ public final class ProtobufUtil {
         delete.addDeleteMarker(cell);
       }
     } else {
-      delete = new Delete(row, timestamp);
+      if (delete == null) {
+        throw new IllegalArgumentException("row cannot be null");
+      }
       for (ColumnValue column: proto.getColumnValueList()) {
         byte[] family = column.getFamily().toByteArray();
         for (QualifierValue qv: column.getQualifierValueList()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7239056c/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 3e133c4..e346c34 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
@@ -129,6 +129,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
@@ -696,6 +697,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
               setException(ResponseConverter.buildException(sizeIOE));
           resultOrExceptionBuilder.setIndex(action.getIndex());
           builder.addResultOrException(resultOrExceptionBuilder.build());
+          if (cellScanner != null) {
+            skipCellsForMutation(action, cellScanner);
+          }
           continue;
         }
         if (action.hasGet()) {
@@ -2239,6 +2243,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         rpcServer.getMetrics().exception(e);
         regionActionResultBuilder.setException(ResponseConverter.buildException(e));
         responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
+        // All Mutations in this RegionAction not executed as we can not see the Region online here
+        // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner
+        // corresponding to these Mutations.
+        if (cellScanner != null) {
+          skipCellsForMutations(regionAction.getActionList(), cellScanner);
+        }
         continue;  // For this region it's a failure.
       }
 
@@ -2296,6 +2306,30 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return responseBuilder.build();
   }
 
+  private void skipCellsForMutations(List<Action> actions, CellScanner cellScanner) {
+    for (Action action : actions) {
+      skipCellsForMutation(action, cellScanner);
+    }
+  }
+
+  private void skipCellsForMutation(Action action, CellScanner cellScanner) {
+    try {
+      if (action.hasMutation()) {
+        MutationProto m = action.getMutation();
+        if (m.hasAssociatedCellCount()) {
+          for (int i = 0; i < m.getAssociatedCellCount(); i++) {
+            cellScanner.advance();
+          }
+        }
+      }
+    } catch (IOException e) {
+      // No need to handle these Individual Muatation level issue. Any way this entire RegionAction
+      // marked as failed as we could not see the Region here. At client side the top level
+      // RegionAction exception will be considered first.
+      LOG.error("Error while skipping Cells in CellScanner for invalid Region Mutations", e);
+    }
+  }
+
   /**
    * Mutate data in a table.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/7239056c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index af3a54e..d295ab2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -35,10 +35,12 @@ import org.apache.commons.logging.LogFactory;
 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.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.codec.KeyValueCodec;
 import org.apache.hadoop.hbase.exceptions.OperationConflictException;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -75,6 +77,8 @@ public class TestMultiParallel {
     //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
+    UTIL.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
+        KeyValueCodec.class.getCanonicalName());
     UTIL.startMiniCluster(slaves);
     Table t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
     UTIL.waitTableEnabled(TEST_TABLE);


[21/32] hbase git commit: HBASE-15209 (compatibility) Disable table in HBaseTestingUtility.truncateTable. (Apekshit)

Posted by sy...@apache.org.
HBASE-15209 (compatibility) Disable table in HBaseTestingUtility.truncateTable. (Apekshit)

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


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

Branch: refs/heads/hbase-12439
Commit: 31f9f2fb67e4c83748d725947ed5eb2cd13df1ef
Parents: 81d81c9
Author: Apekshit <ap...@gmail.com>
Authored: Tue Feb 2 18:24:17 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 20:41:40 2016 -0800

----------------------------------------------------------------------
 .../test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java    | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/31f9f2fb/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 71a3344..4360e1e 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
@@ -1874,6 +1874,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException {
     Admin admin = getHBaseAdmin();
+    if (!admin.isTableDisabled(tableName)) {
+      admin.disableTable(tableName);
+    }
     admin.truncateTable(tableName, preserveRegions);
     return (HTable) getConnection().getTable(tableName);
   }


[13/32] hbase git commit: update zhangduo affiliation

Posted by sy...@apache.org.
update zhangduo affiliation


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

Branch: refs/heads/hbase-12439
Commit: bb71446e1556a9cfd4071f64bab63a52b6ee14c2
Parents: 8f20bc7
Author: zhangduo <zh...@apache.org>
Authored: Fri Feb 5 15:47:29 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Feb 5 15:47:29 2016 +0800

----------------------------------------------------------------------
 pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bb71446e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1169c78..87865d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -500,8 +500,8 @@
       <name>Duo Zhang</name>
       <email>zhangduo@apache.org</email>
       <timezone>+8</timezone>
-      <organization>unaffiliated</organization>
-      <organizationUrl>https://github.com/Apache9</organizationUrl>
+      <organization>Xiaomi</organization>
+      <organizationUrl>http://www.mi.com</organizationUrl>
     </developer>
     <developer>
       <id>zjushch</id>


[29/32] hbase git commit: HBASE-15158 Change order in which we do write pipeline operations; do all under row locks

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index 5fe2061..86a3c3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -127,10 +127,7 @@ class FSWALEntry extends Entry {
       }
     }
 
-    // This has to stay in this order
-    WALKey key = getKey();
-    key.setLogSeqNum(regionSequenceId);
-    key.setWriteEntry(we);
+    getKey().setWriteEntry(we);
     return regionSequenceId;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
index c094ced..7c40323 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
@@ -166,7 +166,7 @@ public class HLogKey extends WALKey implements Writable {
           this.tablename.getName().length, out,
           compressionContext.tableDict);
     }
-    out.writeLong(this.logSeqNum);
+    out.writeLong(getSequenceId());
     out.writeLong(this.writeTime);
     // Don't need to write the clusters information as we are using protobufs from 0.95
     // Writing only the first clusterId for testing the legacy read
@@ -213,7 +213,7 @@ public class HLogKey extends WALKey implements Writable {
       tablenameBytes = Compressor.readCompressed(in, compressionContext.tableDict);
     }
 
-    this.logSeqNum = in.readLong();
+    setSequenceId(in.readLong());
     this.writeTime = in.readLong();
 
     this.clusterIds.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index c89a466..f268422 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -37,9 +37,9 @@ import org.apache.hadoop.hbase.wal.WALKey;
 import com.google.protobuf.TextFormat;
 
 /**
- * Helper methods to ease Region Server integration with the write ahead log.
+ * Helper methods to ease Region Server integration with the Write Ahead Log (WAL).
  * Note that methods in this class specifically should not require access to anything
- * other than the API found in {@link WAL}.
+ * other than the API found in {@link WAL}. For internal use only.
  */
 @InterfaceAudience.Private
 public class WALUtil {
@@ -51,86 +51,108 @@ public class WALUtil {
 
   /**
    * Write the marker that a compaction has succeeded and is about to be committed.
-   * This provides info to the HMaster to allow it to recover the compaction if
-   * this regionserver dies in the middle (This part is not yet implemented). It also prevents
-   * the compaction from finishing if this regionserver has already lost its lease on the log.
+   * This provides info to the HMaster to allow it to recover the compaction if this regionserver
+   * dies in the middle. It also prevents the compaction from finishing if this regionserver has
+   * already lost its lease on the log.
+   *
+   * <p>This write is for internal use only. Not for external client consumption.
    * @param mvcc Used by WAL to get sequence Id for the waledit.
    */
-  public static long writeCompactionMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
+  public static WALKey writeCompactionMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
       final CompactionDescriptor c, MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    long trx = writeMarker(wal, htd, hri, WALEdit.createCompaction(hri, c), mvcc, true);
+    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createCompaction(hri, c), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
     }
-    return trx;
+    return walKey;
   }
 
   /**
    * Write a flush marker indicating a start / abort or a complete of a region flush
+   *
+   * <p>This write is for internal use only. Not for external client consumption.
    */
-  public static long writeFlushMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
+  public static WALKey writeFlushMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
       final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    long trx = writeMarker(wal, htd, hri, WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
+    WALKey walKey =
+      doFullAppendTransaction(wal, htd, hri, WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
     }
-    return trx;
+    return walKey;
   }
 
   /**
-   * Write a region open marker indicating that the region is opened
+   * Write a region open marker indicating that the region is opened.
+   * This write is for internal use only. Not for external client consumption.
    */
-  public static long writeRegionEventMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
+  public static WALKey writeRegionEventMarker(WAL wal, HTableDescriptor htd, HRegionInfo hri,
       final RegionEventDescriptor r, final MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    long trx = writeMarker(wal, htd, hri, WALEdit.createRegionEventWALEdit(hri, r), mvcc, true);
+    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createRegionEventWALEdit(hri, r), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
     }
-    return trx;
+    return walKey;
   }
 
   /**
    * Write a log marker that a bulk load has succeeded and is about to be committed.
-   *
-   * @param wal        The log to write into.
-   * @param htd        A description of the table that we are bulk loading into.
-   * @param hri       A description of the region in the table that we are bulk loading into.
+   * This write is for internal use only. Not for external client consumption.
+   * @param wal The log to write into.
+   * @param htd A description of the table that we are bulk loading into.
+   * @param hri A description of the region in the table that we are bulk loading into.
    * @param desc A protocol buffers based description of the client's bulk loading request
-   * @return txid of this transaction or if nothing to do, the last txid
+   * @return walKey with sequenceid filled out for this bulk load marker
    * @throws IOException We will throw an IOException if we can not append to the HLog.
    */
-  public static long writeBulkLoadMarkerAndSync(final WAL wal, final HTableDescriptor htd,
+  public static WALKey writeBulkLoadMarkerAndSync(final WAL wal, final HTableDescriptor htd,
       final HRegionInfo hri, final WALProtos.BulkLoadDescriptor desc,
       final MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    long trx = writeMarker(wal, htd, hri, WALEdit.createBulkLoadEvent(hri, desc), mvcc, true);
+    WALKey walKey = writeMarker(wal, htd, hri, WALEdit.createBulkLoadEvent(hri, desc), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
     }
-    return trx;
+    return walKey;
   }
 
-  private static long writeMarker(final WAL wal, final HTableDescriptor htd, final HRegionInfo hri,
-      final WALEdit edit, final MultiVersionConcurrencyControl mvcc, final boolean sync)
+  private static WALKey writeMarker(final WAL wal, final HTableDescriptor htd,
+      final HRegionInfo hri, final WALEdit edit, final MultiVersionConcurrencyControl mvcc)
+  throws IOException {
+    // If sync == true in below, then timeout is not used; safe to pass UNSPECIFIED_TIMEOUT
+    return doFullAppendTransaction(wal, htd, hri, edit, mvcc, true);
+  }
+
+  /**
+   * A 'full' WAL transaction involves starting an mvcc transaction followed by an append,
+   * an optional sync, and then a call to complete the mvcc transaction. This method does it all.
+   * Good for case of adding a single edit or marker to the WAL.
+   *
+   * <p>This write is for internal use only. Not for external client consumption.
+   * @return WALKey that was added to the WAL.
+   */
+  public static WALKey doFullAppendTransaction(final WAL wal, final HTableDescriptor htd,
+      final HRegionInfo hri, final WALEdit edit, final MultiVersionConcurrencyControl mvcc,
+      final boolean sync)
   throws IOException {
     // TODO: Pass in current time to use?
-    WALKey key =
-      new HLogKey(hri.getEncodedNameAsBytes(), hri.getTable(), System.currentTimeMillis(), mvcc);
-    // Add it to the log but the false specifies that we don't need to add it to the memstore
+    WALKey walKey =
+      new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(), System.currentTimeMillis(), mvcc);
     long trx = MultiVersionConcurrencyControl.NONE;
     try {
-      trx = wal.append(htd, hri, key, edit, false);
-      if (sync) wal.sync(trx);
-    } finally {
-      // If you get hung here, is it a real WAL or a mocked WAL? If the latter, you need to
-      // trip the latch that is inside in getWriteEntry up in your mock. See down in the append
-      // called from onEvent in FSHLog.
-      MultiVersionConcurrencyControl.WriteEntry we = key.getWriteEntry();
-      if (mvcc != null && we != null) mvcc.complete(we);
+      trx = wal.append(htd, hri, walKey, edit, false);
+      if (sync) {
+        wal.sync(trx);
+      }
+      // Call complete only here because these are markers only. They are not for clients to read.
+      mvcc.complete(walKey.getWriteEntry());
+    } catch (IOException ioe) {
+      mvcc.complete(walKey.getWriteEntry());
+      throw ioe;
     }
-    return trx;
+    return walKey;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index 4091a82..09096fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -30,52 +30,50 @@ import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FamilyScope;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.ScopeType;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.SequenceId;
+// imports for things that haven't moved from regionserver.wal yet.
+import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
+import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
 
-// imports for things that haven't moved from regionserver.wal yet.
-import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
-import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
-
 /**
- * A Key for an entry in the change log.
+ * A Key for an entry in the WAL.
  *
  * The log intermingles edits to many tables and rows, so each log entry
  * identifies the appropriate table and row.  Within a table and row, they're
  * also sorted.
  *
- * <p>Some Transactional edits (START, COMMIT, ABORT) will not have an
- * associated row.
+ * <p>Some Transactional edits (START, COMMIT, ABORT) will not have an associated row.
  *
  * Note that protected members marked @InterfaceAudience.Private are only protected
  * to support the legacy HLogKey class, which is in a different package.
- * 
- * <p>
  */
 // TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
 //       purposes. They need to be merged into WALEntry.
-// TODO: Cleanup. We have logSeqNum and then WriteEntry, both are sequence id'ing. Fix.
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
 public class WALKey implements SequenceId, Comparable<WALKey> {
   private static final Log LOG = LogFactory.getLog(WALKey.class);
+  private final CountDownLatch sequenceIdAssignedLatch = new CountDownLatch(1);
+  /**
+   * Used to represent when a particular wal key doesn't know/care about the sequence ordering.
+   */
+  public static final long NO_SEQUENCE_ID = -1;
 
   @InterfaceAudience.Private // For internal use only.
   public MultiVersionConcurrencyControl getMvcc() {
@@ -83,25 +81,22 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   }
 
   /**
-   * Will block until a write entry has been assigned by they WAL subsystem.
-   * @return A WriteEntry gotten from local WAL subsystem. Must be completed by calling
-   *     {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
-   *     or
-   *     {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
+   * Use it to complete mvcc transaction. This WALKey was part of
+   * (the transaction is started when you call append; see the comment on FSHLog#append). To
+   * complete call
+   * {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
+   * or {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
+   * @return A WriteEntry gotten from local WAL subsystem.
    * @see #setWriteEntry(MultiVersionConcurrencyControl.WriteEntry)
    */
   @InterfaceAudience.Private // For internal use only.
   public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException {
     try {
-      this.seqNumAssignedLatch.await();
+      this.sequenceIdAssignedLatch.await();
     } catch (InterruptedException ie) {
-      // If interrupted... clear out our entry else we can block up mvcc.
       MultiVersionConcurrencyControl mvcc = getMvcc();
-      LOG.debug("mvcc=" + mvcc + ", writeEntry=" + this.writeEntry);
-      if (mvcc != null) {
-        if (this.writeEntry != null) {
-          mvcc.complete(this.writeEntry);
-        }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("mvcc=" + mvcc + ", writeEntry=" + this.writeEntry);
       }
       InterruptedIOException iie = new InterruptedIOException();
       iie.initCause(ie);
@@ -112,11 +107,19 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
 
   @InterfaceAudience.Private // For internal use only.
   public void setWriteEntry(MultiVersionConcurrencyControl.WriteEntry writeEntry) {
+    if (this.writeEntry != null) {
+      throw new RuntimeException("Non-null!!!");
+    }
     this.writeEntry = writeEntry;
-    this.seqNumAssignedLatch.countDown();
+    // Set our sequenceid now using WriteEntry.
+    if (this.writeEntry != null) {
+      this.sequenceId = this.writeEntry.getWriteNumber();
+    }
+    this.sequenceIdAssignedLatch.countDown();
   }
 
-  // should be < 0 (@see HLogKey#readFields(DataInput))
+  // REMOVE!!!! No more Writables!!!!
+  // Should be < 0 (@see HLogKey#readFields(DataInput))
   // version 2 supports WAL compression
   // public members here are only public because of HLogKey
   @InterfaceAudience.Private
@@ -163,21 +166,23 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   @InterfaceAudience.Private
   protected static final Version VERSION = Version.COMPRESSED;
 
-  /** Used to represent when a particular wal key doesn't know/care about the sequence ordering. */
-  public static final long NO_SEQUENCE_ID = -1;
-
-
   // visible for deprecated HLogKey
   @InterfaceAudience.Private
   protected byte [] encodedRegionName;
   // visible for deprecated HLogKey
   @InterfaceAudience.Private
   protected TableName tablename;
-  // visible for deprecated HLogKey
-  @InterfaceAudience.Private
-  protected long logSeqNum;
+  /**
+   * SequenceId for this edit. Set post-construction at write-to-WAL time. Until then it is
+   * NO_SEQUENCE_ID. Change it so multiple threads can read it -- e.g. access is synchronized.
+   */
+  private long sequenceId;
+
+  /**
+   * Used during WAL replay; the sequenceId of the edit when it came into the system.
+   */
   private long origLogSeqNum = 0;
-  private CountDownLatch seqNumAssignedLatch = new CountDownLatch(1);
+
   // Time at which this edit was written.
   // visible for deprecated HLogKey
   @InterfaceAudience.Private
@@ -193,6 +198,9 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   private long nonceGroup = HConstants.NO_NONCE;
   private long nonce = HConstants.NO_NONCE;
   private MultiVersionConcurrencyControl mvcc;
+  /**
+   * Set in a way visible to multiple threads; e.g. synchronized getter/setters.
+   */
   private MultiVersionConcurrencyControl.WriteEntry writeEntry;
   public static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
 
@@ -215,10 +223,15 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         HConstants.NO_NONCE, HConstants.NO_NONCE, null);
   }
 
+  /**
+   * @deprecated Remove. Useless.
+   */
+  @Deprecated // REMOVE
   public WALKey(final byte[] encodedRegionName, final TableName tablename) {
     this(encodedRegionName, tablename, System.currentTimeMillis());
   }
 
+  // TODO: Fix being able to pass in sequenceid.
   public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
     init(encodedRegionName,
         tablename,
@@ -257,6 +270,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    * @param now               Time at which this edit was written.
    * @param clusterIds        the clusters that have consumed the change(used in Replication)
    */
+  // TODO: Fix being able to pass in sequenceid.
   public WALKey(final byte[] encodedRegionName,
                 final TableName tablename,
                 long logSeqNum,
@@ -300,6 +314,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
    * @param nonceGroup
    * @param nonce
    */
+  // TODO: Fix being able to pass in sequenceid.
   public WALKey(final byte[] encodedRegionName,
                 final TableName tablename,
                 long logSeqNum,
@@ -325,7 +340,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
                       long nonceGroup,
                       long nonce,
                       MultiVersionConcurrencyControl mvcc) {
-    this.logSeqNum = logSeqNum;
+    this.sequenceId = logSeqNum;
     this.writeTime = now;
     this.clusterIds = clusterIds;
     this.encodedRegionName = encodedRegionName;
@@ -333,6 +348,15 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     this.nonceGroup = nonceGroup;
     this.nonce = nonce;
     this.mvcc = mvcc;
+    if (logSeqNum != NO_SEQUENCE_ID) {
+      setSequenceId(logSeqNum);
+    }
+  }
+
+  // For HLogKey and deserialization. DO NOT USE. See setWriteEntry below.
+  @InterfaceAudience.Private
+  protected void setSequenceId(long sequenceId) {
+    this.sequenceId = sequenceId;
   }
 
   /**
@@ -352,32 +376,24 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     return tablename;
   }
 
-  /** @return log sequence number */
-  public long getLogSeqNum() {
-    return this.logSeqNum;
-  }
-
-  /**
-   * Allow that the log sequence id to be set post-construction
-   * Only public for org.apache.hadoop.hbase.regionserver.wal.FSWALEntry
-   * @param sequence
+  /** @return log sequence number
+   * @deprecated Use {@link #getSequenceId()}
    */
-  @InterfaceAudience.Private
-  public void setLogSeqNum(final long sequence) {
-    this.logSeqNum = sequence;
-
+  @Deprecated
+  public long getLogSeqNum() {
+    return getSequenceId();
   }
 
   /**
-   * Used to set original seq Id for WALKey during wal replay
-   * @param seqId
+   * Used to set original sequenceId for WALKey during WAL replay
    */
-  public void setOrigLogSeqNum(final long seqId) {
-    this.origLogSeqNum = seqId;
+  public void setOrigLogSeqNum(final long sequenceId) {
+    this.origLogSeqNum = sequenceId;
   }
   
   /**
-   * Return a positive long if current WALKey is created from a replay edit
+   * Return a positive long if current WALKey is created from a replay edit; a replay edit is an
+   * edit that came in when replaying WALs of a crashed server.
    * @return original sequence number of the WALEdit
    */
   public long getOrigLogSeqNum() {
@@ -385,43 +401,14 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   }
   
   /**
-   * Wait for sequence number to be assigned &amp; return the assigned value
+   * SequenceId is only available post WAL-assign. Calls before this will get you a
+   * {@link #NO_SEQUENCE_ID}. See the comment on FSHLog#append and #getWriteNumber in this method
+   * for more on when this sequenceId comes available.
    * @return long the new assigned sequence number
-   * @throws IOException
    */
   @Override
-  public long getSequenceId() throws IOException {
-    return getSequenceId(-1);
-  }
-
-  /**
-   * Wait for sequence number to be assigned &amp; return the assigned value.
-   * @param maxWaitForSeqId maximum time to wait in milliseconds for sequenceid
-   * @return long the new assigned sequence number
-   * @throws IOException
-   */
-  public long getSequenceId(final long maxWaitForSeqId) throws IOException {
-    // TODO: This implementation waiting on a latch is problematic because if a higher level
-    // determines we should stop or abort, there is no global list of all these blocked WALKeys
-    // waiting on a sequence id; they can't be cancelled... interrupted. See getNextSequenceId.
-    //
-    // UPDATE: I think we can remove the timeout now we are stamping all walkeys with sequenceid,
-    // even those that have failed (previously we were not... so they would just hang out...).
-    // St.Ack 20150910
-    try {
-      if (maxWaitForSeqId < 0) {
-        this.seqNumAssignedLatch.await();
-      } else if (!this.seqNumAssignedLatch.await(maxWaitForSeqId, TimeUnit.MILLISECONDS)) {
-        throw new TimeoutIOException("Failed to get sequenceid after " + maxWaitForSeqId +
-          "ms; WAL system stuck or has gone away?");
-      }
-    } catch (InterruptedException ie) {
-      LOG.warn("Thread interrupted waiting for next log sequence number");
-      InterruptedIOException iie = new InterruptedIOException();
-      iie.initCause(ie);
-      throw iie;
-    }
-    return this.logSeqNum;
+  public long getSequenceId() {
+    return this.sequenceId;
   }
 
   /**
@@ -495,7 +482,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   @Override
   public String toString() {
     return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
-      logSeqNum;
+      sequenceId;
   }
 
   /**
@@ -509,7 +496,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
     Map<String, Object> stringMap = new HashMap<String, Object>();
     stringMap.put("table", tablename);
     stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
-    stringMap.put("sequence", logSeqNum);
+    stringMap.put("sequence", getSequenceId());
     return stringMap;
   }
 
@@ -527,7 +514,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   @Override
   public int hashCode() {
     int result = Bytes.hashCode(this.encodedRegionName);
-    result ^= this.logSeqNum;
+    result ^= getSequenceId();
     result ^= this.writeTime;
     return result;
   }
@@ -536,9 +523,11 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
   public int compareTo(WALKey o) {
     int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName);
     if (result == 0) {
-      if (this.logSeqNum < o.logSeqNum) {
+      long sid = getSequenceId();
+      long otherSid = o.getSequenceId();
+      if (sid < otherSid) {
         result = -1;
-      } else if (this.logSeqNum  > o.logSeqNum) {
+      } else if (sid  > otherSid) {
         result = 1;
       }
       if (result == 0) {
@@ -592,7 +581,7 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
       builder.setTableName(compressor.compress(this.tablename.getName(),
           compressionContext.tableDict));
     }
-    builder.setLogSequenceNumber(this.logSeqNum);
+    builder.setLogSequenceNumber(getSequenceId());
     builder.setWriteTime(writeTime);
     if (this.origLogSeqNum > 0) {
       builder.setOrigSequenceNumber(this.origLogSeqNum);
@@ -658,10 +647,10 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
         this.scopes.put(family, scope.getScopeType().getNumber());
       }
     }
-    this.logSeqNum = walKey.getLogSequenceNumber();
+    setSequenceId(walKey.getLogSequenceNumber());
     this.writeTime = walKey.getWriteTime();
     if(walKey.hasOrigSequenceNumber()) {
       this.origLogSeqNum = walKey.getOrigSequenceNumber();
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 60bc155..7add8a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -4765,12 +4765,12 @@ public class TestHRegion {
     durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 5000, true, false, true);
 
     // expect skip wal cases
-    durabilityTest(method, Durability.SYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
-    durabilityTest(method, Durability.FSYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
-    durabilityTest(method, Durability.ASYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
-    durabilityTest(method, Durability.SKIP_WAL, Durability.SKIP_WAL, 0, true, false, false);
-    durabilityTest(method, Durability.USE_DEFAULT, Durability.SKIP_WAL, 0, true, false, false);
-    durabilityTest(method, Durability.SKIP_WAL, Durability.USE_DEFAULT, 0, true, false, false);
+    durabilityTest(method, Durability.SYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
+    durabilityTest(method, Durability.FSYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
+    durabilityTest(method, Durability.ASYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
+    durabilityTest(method, Durability.SKIP_WAL, Durability.SKIP_WAL, 0, false, false, false);
+    durabilityTest(method, Durability.USE_DEFAULT, Durability.SKIP_WAL, 0, false, false, false);
+    durabilityTest(method, Durability.SKIP_WAL, Durability.USE_DEFAULT, 0, false, false, false);
 
   }
 


[30/32] hbase git commit: HBASE-15158 Change order in which we do write pipeline operations; do all under row locks

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ec92a8a7/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 f03c205..ac846b6 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
@@ -1,5 +1,4 @@
 /*
- *
  * 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
@@ -29,6 +28,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -41,6 +41,7 @@ import java.util.NavigableSet;
 import java.util.RandomAccess;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ConcurrentHashMap;
@@ -69,7 +70,6 @@ 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.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
@@ -77,7 +77,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
@@ -93,7 +92,6 @@ import org.apache.hadoop.hbase.ShareableMemory;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
-import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
@@ -112,7 +110,7 @@ import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
@@ -123,8 +121,6 @@ import org.apache.hadoop.hbase.filter.FilterWrapper;
 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
 import org.apache.hadoop.hbase.ipc.RpcCallContext;
@@ -148,6 +144,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.Stor
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -169,7 +166,6 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HashedBytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -199,6 +195,7 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.TextFormat;
 
+@SuppressWarnings("deprecation")
 @InterfaceAudience.Private
 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
   private static final Log LOG = LogFactory.getLog(HRegion.class);
@@ -207,18 +204,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     "hbase.hregion.scan.loadColumnFamiliesOnDemand";
 
   /**
-   * Longest time we'll wait on a sequenceid.
-   * Sequenceid comes up out of the WAL subsystem. WAL subsystem can go bad or a test might use
-   * it without cleaning up previous usage properly; generally, a WAL roll is needed. The timeout
-   * is for a latch in WALKey. There is no global accounting of outstanding WALKeys; intentionally
-   * to avoid contention, but it makes it so if an abort or problem, we could be stuck waiting
-   * on the WALKey latch. Revisit.
-   */
-  private final int maxWaitForSeqId;
-  private static final String MAX_WAIT_FOR_SEQ_ID_KEY = "hbase.hregion.max.wait.for.sequenceid.ms";
-  private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 30000;
-
-  /**
    * This is the global default value for durability. All tables/mutations not
    * defining a durability or using USE_DEFAULT will default to this value.
    */
@@ -282,7 +267,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final Counter checkAndMutateChecksPassed = new Counter();
   final Counter checkAndMutateChecksFailed = new Counter();
 
-  //Number of requests
+  // Number of requests
   final Counter readRequestsCount = new Counter();
   final Counter filteredReadRequestsCount = new Counter();
   final Counter writeRequestsCount = new Counter();
@@ -357,7 +342,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   private boolean disallowWritesInRecovering = false;
 
-  // when a region is in recovering state, it can only accept writes not reads
+  // When a region is in recovering state, it can only accept writes not reads
   private volatile boolean recovering = false;
 
   private volatile Optional<ConfigurationManager> configurationManager;
@@ -374,7 +359,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // We achieve this by synchronizing on the scannerReadPoints object.
     synchronized(scannerReadPoints) {
       minimumReadPoint = mvcc.getReadPoint();
-
       for (Long readPoint: this.scannerReadPoints.values()) {
         if (readPoint < minimumReadPoint) {
           minimumReadPoint = readPoint;
@@ -674,7 +658,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
                     DEFAULT_ROWLOCK_WAIT_DURATION);
 
-    this.maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
     this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
     this.htableDescriptor = htd;
     this.rsServices = rsServices;
@@ -1183,7 +1166,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   public void setRecovering(boolean newState) {
     boolean wasRecovering = this.recovering;
-    // before we flip the recovering switch (enabling reads) we should write the region open
+    // Before we flip the recovering switch (enabling reads) we should write the region open
     // event to WAL if needed
     if (wal != null && getRegionServerServices() != null && !writestate.readOnly
         && wasRecovering && !newState) {
@@ -2051,7 +2034,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   /**
    * Should the store be flushed because it is old enough.
    * <p>
-   * Every FlushPolicy should call this to determine whether a store is old enough to flush(except
+   * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
    * that you always flush all stores). Otherwise the {@link #shouldFlush()} method will always
    * returns true which will make a lot of flush requests.
    */
@@ -2152,19 +2135,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * for say installing a bulk loaded file just ahead of the last hfile that was
    * the result of this flush, etc.
    *
-   * @param wal
-   *          Null if we're NOT to go via wal.
-   * @param myseqid
-   *          The seqid to use if <code>wal</code> is null writing out flush
-   *          file.
-   * @param storesToFlush
-   *          The list of stores to flush.
+   * @param wal Null if we're NOT to go via wal.
+   * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file.
+   * @param storesToFlush The list of stores to flush.
    * @return object describing the flush's state
-   * @throws IOException
-   *           general io exceptions
-   * @throws DroppedSnapshotException
-   *           Thrown when replay of wal is required because a Snapshot was not
-   *           properly persisted.
+   * @throws IOException general io exceptions
+   * @throws DroppedSnapshotException Thrown when replay of WAL is required.
    */
   protected FlushResult internalFlushcache(final WAL wal, final long myseqid,
       final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
@@ -2188,65 +2164,48 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       throw new IOException("Aborting flush because server is aborted...");
     }
     final long startTime = EnvironmentEdgeManager.currentTime();
-    // If nothing to flush, return, but we need to safely update the region sequence id
+    // If nothing to flush, return, but return with a valid unused sequenceId.
+    // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a
+    // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
+    // to no other that it can use to associate with the bulk load. Hence this little dance below
+    // to go get one.
     if (this.memstoreSize.get() <= 0) {
-      // Take an update lock because am about to change the sequence id and we want the sequence id
-      // to be at the border of the empty memstore.
-      MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
+      // Take an update lock so no edits can come into memory just yet.
       this.updatesLock.writeLock().lock();
+      WriteEntry writeEntry = null;
       try {
         if (this.memstoreSize.get() <= 0) {
           // Presume that if there are still no edits in the memstore, then there are no edits for
           // this region out in the WAL subsystem so no need to do any trickery clearing out
-          // edits in the WAL system. Up the sequence number so the resulting flush id is for
-          // sure just beyond the last appended region edit (useful as a marker when bulk loading,
-          // etc.). NOTE: The writeEntry write number is NOT in the WAL.. there is no WAL writing
-          // here.
+          // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
+          // sure just beyond the last appended region edit and not associated with any edit
+          // (useful as marker when bulk loading, etc.).
+          FlushResult flushResult = null;
           if (wal != null) {
             writeEntry = mvcc.begin();
             long flushOpSeqId = writeEntry.getWriteNumber();
-            FlushResult flushResult = new FlushResultImpl(
-                FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
-                flushOpSeqId,
-                "Nothing to flush",
-                writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
-            // TODO: Lets see if we hang here, if there is a scenario where an outstanding reader
-            // with a read point is in advance of this write point.
+            flushResult = new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
+              flushOpSeqId, "Nothing to flush",
+            writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
             mvcc.completeAndWait(writeEntry);
+            // Set to null so we don't complete it again down in finally block.
             writeEntry = null;
             return new PrepareFlushResult(flushResult, myseqid);
           } else {
-            return new PrepareFlushResult(
-              new FlushResultImpl(
-                  FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
-                  "Nothing to flush",
-                  false),
-              myseqid);
+            return new PrepareFlushResult(new FlushResultImpl(
+              FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid);
           }
         }
       } finally {
-        this.updatesLock.writeLock().unlock();
         if (writeEntry != null) {
+          // If writeEntry is non-null, this operation failed; the mvcc transaction failed...
+          // but complete it anyways so it doesn't block the mvcc queue.
           mvcc.complete(writeEntry);
         }
+        this.updatesLock.writeLock().unlock();
       }
     }
-
-    if (LOG.isInfoEnabled()) {
-      // Log a fat line detailing what is being flushed.
-      StringBuilder perCfExtras = null;
-      if (!isAllFamilies(storesToFlush)) {
-        perCfExtras = new StringBuilder();
-        for (Store store: storesToFlush) {
-          perCfExtras.append("; ").append(store.getColumnFamilyName());
-          perCfExtras.append("=").append(StringUtils.byteDesc(store.getMemStoreSize()));
-        }
-      }
-      LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
-        " column families, memstore=" + StringUtils.byteDesc(this.memstoreSize.get()) +
-        ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
-        ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + myseqid));
-    }
+    logFatLineOnFlush(storesToFlush, myseqid);
     // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
     // to do this for a moment.  It is quick. We also set the memstore size to zero here before we
     // allow updates again so its value will represent the size of the updates received
@@ -2257,8 +2216,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     status.setStatus("Obtaining lock to block concurrent updates");
     // block waiting for the lock for internal flush
     this.updatesLock.writeLock().lock();
-    status.setStatus("Preparing to flush by snapshotting stores in " +
-      getRegionInfo().getEncodedName());
+    status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
     long totalFlushableSizeOfFlushableStores = 0;
 
     Set<byte[]> flushedFamilyNames = new HashSet<byte[]>();
@@ -2280,109 +2238,117 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // will be in advance of this sequence id.
     long flushedSeqId = HConstants.NO_SEQNUM;
     byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
-
-    long trxId = 0;
-    MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.begin();
     try {
-      try {
-        if (wal != null) {
-          Long earliestUnflushedSequenceIdForTheRegion =
+      if (wal != null) {
+        Long earliestUnflushedSequenceIdForTheRegion =
             wal.startCacheFlush(encodedRegionName, flushedFamilyNames);
-          if (earliestUnflushedSequenceIdForTheRegion == null) {
-            // This should never happen. This is how startCacheFlush signals flush cannot proceed.
-            String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
-            status.setStatus(msg);
-            return new PrepareFlushResult(
+        if (earliestUnflushedSequenceIdForTheRegion == null) {
+          // This should never happen. This is how startCacheFlush signals flush cannot proceed.
+          String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
+          status.setStatus(msg);
+          return new PrepareFlushResult(
               new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
               myseqid);
-          }
-          flushOpSeqId = getNextSequenceId(wal);
-          // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
-          flushedSeqId =
-            earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
-              flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
-        } else {
-          // use the provided sequence Id as WAL is not being used for this flush.
-          flushedSeqId = flushOpSeqId = myseqid;
         }
+        flushOpSeqId = getNextSequenceId(wal);
+        // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
+        flushedSeqId =
+            earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
+                flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
+      } else {
+        // use the provided sequence Id as WAL is not being used for this flush.
+        flushedSeqId = flushOpSeqId = myseqid;
+      }
 
-        for (Store s : storesToFlush) {
-          totalFlushableSizeOfFlushableStores += s.getFlushableSize();
-          storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
-          committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
-          storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize());
-        }
+      for (Store s : storesToFlush) {
+        totalFlushableSizeOfFlushableStores += s.getFlushableSize();
+        storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
+        committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
+        storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize());
+      }
 
-        // write the snapshot start to WAL
-        if (wal != null && !writestate.readOnly) {
-          FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
+      // write the snapshot start to WAL
+      if (wal != null && !writestate.readOnly) {
+        FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
             getRegionInfo(), flushOpSeqId, committedFiles);
-          // no sync. Sync is below where we do not hold the updates lock
-          trxId = WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
-            desc, false, mvcc);
-        }
-
-        // Prepare flush (take a snapshot)
-        for (StoreFlushContext flush : storeFlushCtxs.values()) {
-          flush.prepare();
-        }
-      } catch (IOException ex) {
-        if (wal != null) {
-          if (trxId > 0) { // check whether we have already written START_FLUSH to WAL
-            try {
-              FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
-                getRegionInfo(), flushOpSeqId, committedFiles);
-              WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
-                desc, false, mvcc);
-            } catch (Throwable t) {
-              LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" +
-                  StringUtils.stringifyException(t));
-              // ignore this since we will be aborting the RS with DSE.
-            }
-          }
-          // we have called wal.startCacheFlush(), now we have to abort it
-          wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
-          throw ex; // let upper layers deal with it.
-        }
-      } finally {
-        this.updatesLock.writeLock().unlock();
-      }
-      String s = "Finished memstore snapshotting " + this +
-        ", syncing WAL and waiting on mvcc, flushsize=" + totalFlushableSizeOfFlushableStores;
-      status.setStatus(s);
-      if (LOG.isTraceEnabled()) LOG.trace(s);
-      // sync unflushed WAL changes
-      // see HBASE-8208 for details
-      if (wal != null) {
-        try {
-          wal.sync(); // ensure that flush marker is sync'ed
-        } catch (IOException ioe) {
-          wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
-          throw ioe;
-        }
+        // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
+        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false, mvcc);
       }
 
-      // wait for all in-progress transactions to commit to WAL before
-      // we can start the flush. This prevents
-      // uncommitted transactions from being written into HFiles.
-      // We have to block before we start the flush, otherwise keys that
-      // were removed via a rollbackMemstore could be written to Hfiles.
-      mvcc.completeAndWait(writeEntry);
-      // set writeEntry to null to prevent mvcc.complete from being called again inside finally
-      // block
-      writeEntry = null;
-    } finally {
-      if (writeEntry != null) {
-        // In case of failure just mark current writeEntry as complete.
-        mvcc.complete(writeEntry);
+      // Prepare flush (take a snapshot)
+      for (StoreFlushContext flush : storeFlushCtxs.values()) {
+        flush.prepare();
       }
+    } catch (IOException ex) {
+      doAbortFlushToWAL(wal, flushOpSeqId, committedFiles);
+      throw ex;
+    } finally {
+      this.updatesLock.writeLock().unlock();
     }
+    String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " +
+        "flushsize=" + totalFlushableSizeOfFlushableStores;
+    status.setStatus(s);
+    doSyncOfUnflushedWALChanges(wal, getRegionInfo());
     return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
         flushOpSeqId, flushedSeqId, totalFlushableSizeOfFlushableStores);
   }
 
   /**
-   * @param families
+   * Utility method broken out of internalPrepareFlushCache so that method is smaller.
+   */
+  private void logFatLineOnFlush(final Collection<Store> storesToFlush, final long sequenceId) {
+    if (!LOG.isInfoEnabled()) {
+      return;
+    }
+    // Log a fat line detailing what is being flushed.
+    StringBuilder perCfExtras = null;
+    if (!isAllFamilies(storesToFlush)) {
+      perCfExtras = new StringBuilder();
+      for (Store store: storesToFlush) {
+        perCfExtras.append("; ").append(store.getColumnFamilyName());
+        perCfExtras.append("=").append(StringUtils.byteDesc(store.getMemStoreSize()));
+      }
+    }
+    LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
+        " column families, memstore=" + StringUtils.byteDesc(this.memstoreSize.get()) +
+        ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
+        ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
+  }
+
+  private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId,
+      final Map<byte[], List<Path>> committedFiles) {
+    if (wal == null) return;
+    try {
+      FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
+          getRegionInfo(), flushOpSeqId, committedFiles);
+      WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false,
+          mvcc);
+    } catch (Throwable t) {
+      LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" +
+          StringUtils.stringifyException(t));
+      // ignore this since we will be aborting the RS with DSE.
+    }
+    // we have called wal.startCacheFlush(), now we have to abort it
+    wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
+  }
+
+  /**
+   * Sync unflushed WAL changes. See HBASE-8208 for details
+   */
+  private static void doSyncOfUnflushedWALChanges(final WAL wal, final HRegionInfo hri)
+  throws IOException {
+    if (wal == null) {
+      return;
+    }
+    try {
+      wal.sync(); // ensure that flush marker is sync'ed
+    } catch (IOException ioe) {
+      wal.abortCacheFlush(hri.getEncodedNameAsBytes());
+      throw ioe;
+    }
+  }
+
+  /**
    * @return True if passed Set is all families in the region.
    */
   private boolean isAllFamilies(final Collection<Store> families) {
@@ -2400,8 +2366,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
         getRegionInfo(), -1, new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR));
       try {
-        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
-          desc, true, mvcc);
+        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, true, mvcc);
         return true;
       } catch (IOException e) {
         LOG.warn(getRegionInfo().getEncodedName() + " : "
@@ -2471,8 +2436,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
         FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
           getRegionInfo(), flushOpSeqId, committedFiles);
-        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
-          desc, true, mvcc);
+        WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, true, mvcc);
       }
     } catch (Throwable t) {
       // An exception here means that the snapshot was not persisted.
@@ -2485,8 +2449,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         try {
           FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
             getRegionInfo(), flushOpSeqId, committedFiles);
-          WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
-            desc, false, mvcc);
+          WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(), desc, false, mvcc);
         } catch (Throwable ex) {
           LOG.warn(getRegionInfo().getEncodedName() + " : "
               + "failed writing ABORT_FLUSH marker to WAL", ex);
@@ -2557,15 +2520,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   @VisibleForTesting
   protected long getNextSequenceId(final WAL wal) throws IOException {
-    // TODO: For review. Putting an empty edit in to get a sequenceid out will not work if the
-    // WAL is banjaxed... if it has gotten an exception and the WAL has not yet been rolled or
-    // aborted. In this case, we'll just get stuck here. For now, until HBASE-12751, just have
-    // a timeout. May happen in tests after we tightened the semantic via HBASE-14317.
-    // Also, the getSequenceId blocks on a latch. There is no global list of outstanding latches
-    // so if an abort or stop, there is no way to call them in.
-    WALKey key = this.appendEmptyEdit(wal);
-    mvcc.complete(key.getWriteEntry());
-    return key.getSequenceId(this.maxWaitForSeqId);
+    WriteEntry we = mvcc.begin();
+    mvcc.completeAndWait(we);
+    return we.getWriteNumber();
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -2754,13 +2711,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * accumulating status codes and tracking the index at which processing
    * is proceeding.
    */
-  private abstract static class BatchOperationInProgress<T> {
+  private abstract static class BatchOperation<T> {
     T[] operations;
     int nextIndexToProcess = 0;
     OperationStatus[] retCodeDetails;
     WALEdit[] walEditsFromCoprocessors;
 
-    public BatchOperationInProgress(T[] operations) {
+    public BatchOperation(T[] operations) {
       this.operations = operations;
       this.retCodeDetails = new OperationStatus[operations.length];
       this.walEditsFromCoprocessors = new WALEdit[operations.length];
@@ -2780,7 +2737,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  private static class MutationBatch extends BatchOperationInProgress<Mutation> {
+  private static class MutationBatch extends BatchOperation<Mutation> {
     private long nonceGroup;
     private long nonce;
     public MutationBatch(Mutation[] operations, long nonceGroup, long nonce) {
@@ -2820,7 +2777,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  private static class ReplayBatch extends BatchOperationInProgress<MutationReplay> {
+  private static class ReplayBatch extends BatchOperation<MutationReplay> {
     private long replaySeqId = 0;
     public ReplayBatch(MutationReplay[] operations, long seqId) {
       super(operations);
@@ -2906,7 +2863,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    *         OperationStatusCode and the exceptionMessage if any.
    * @throws IOException
    */
-  OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
+  OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException {
     boolean initialized = false;
     Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
     startRegionOperation(op);
@@ -2920,11 +2877,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (!initialized) {
           this.writeRequestsCount.add(batchOp.operations.length);
           if (!batchOp.isInReplay()) {
-            doPreMutationHook(batchOp);
+            doPreBatchMutateHook(batchOp);
           }
           initialized = true;
         }
-        long addedSize = doMiniBatchMutation(batchOp);
+        long addedSize = doMiniBatchMutate(batchOp);
         long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
         if (isFlushSize(newSize)) {
           requestFlush();
@@ -2936,8 +2893,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return batchOp.retCodeDetails;
   }
 
-
-  private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
+  private void doPreBatchMutateHook(BatchOperation<?> batchOp)
       throws IOException {
     /* Run coprocessor pre hook outside of locks to avoid deadlock */
     WALEdit walEdit = new WALEdit();
@@ -2976,103 +2932,58 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  /**
+   * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[], long, long)}
+   * In here we also handle replay of edits on region recover.
+   * @return Change in size brought about by applying <code>batchOp</code>
+   */
   @SuppressWarnings("unchecked")
-  private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
-    boolean isInReplay = batchOp.isInReplay();
-    // variable to note if all Put items are for the same CF -- metrics related
+  // TODO: This needs a rewrite. Doesn't have to be this long. St.Ack 20160120
+  private long doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
+    boolean replay = batchOp.isInReplay();
+    // Variable to note if all Put items are for the same CF -- metrics related
     boolean putsCfSetConsistent = true;
-    //The set of columnFamilies first seen for Put.
-    Set<byte[]> putsCfSet = null;
-    // variable to note if all Delete items are for the same CF -- metrics related
+    // Variable to note if all Delete items are for the same CF -- metrics related
     boolean deletesCfSetConsistent = true;
-    //The set of columnFamilies first seen for Delete.
+    // The set of columnFamilies first seen for Put.
+    Set<byte[]> putsCfSet = null;
+    // The set of columnFamilies first seen for Delete.
     Set<byte[]> deletesCfSet = null;
-
-    long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
-    WALEdit walEdit = new WALEdit(isInReplay);
-    MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
-    long txid = 0;
-    boolean doRollBackMemstore = false;
+    long currentNonceGroup = HConstants.NO_NONCE;
+    long currentNonce = HConstants.NO_NONCE;
+    WALEdit walEdit = new WALEdit(replay);
     boolean locked = false;
-
-    /** Keep track of the locks we hold so we can release them in finally clause */
-    List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
     // reference family maps directly so coprocessors can mutate them if desired
     Map<byte[], List<Cell>>[] familyMaps = new Map[batchOp.operations.length];
     // We try to set up a batch in the range [firstIndex,lastIndexExclusive)
     int firstIndex = batchOp.nextIndexToProcess;
     int lastIndexExclusive = firstIndex;
     boolean success = false;
-    int noOfPuts = 0, noOfDeletes = 0;
-    WALKey walKey = null;
-    long mvccNum = 0;
+    int noOfPuts = 0;
+    int noOfDeletes = 0;
+    WriteEntry writeEntry = null;
+    /** Keep track of the locks we hold so we can release them in finally clause */
+    List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
     try {
-      // ------------------------------------
-      // STEP 1. Try to acquire as many locks as we can, and ensure
-      // we acquire at least one.
-      // ----------------------------------
+      // STEP 1. Try to acquire as many locks as we can, and ensure we acquire at least one.
       int numReadyToWrite = 0;
       long now = EnvironmentEdgeManager.currentTime();
       while (lastIndexExclusive < batchOp.operations.length) {
-        Mutation mutation = batchOp.getMutation(lastIndexExclusive);
-        boolean isPutMutation = mutation instanceof Put;
-
-        Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
-        // store the family map reference to allow for mutations
-        familyMaps[lastIndexExclusive] = familyMap;
-
-        // skip anything that "ran" already
-        if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
-            != OperationStatusCode.NOT_RUN) {
-          lastIndexExclusive++;
-          continue;
-        }
-
-        try {
-          if (isPutMutation) {
-            // Check the families in the put. If bad, skip this one.
-            if (isInReplay) {
-              removeNonExistentColumnFamilyForReplay(familyMap);
-            } else {
-              checkFamilies(familyMap.keySet());
-            }
-            checkTimestamps(mutation.getFamilyCellMap(), now);
-          } else {
-            prepareDelete((Delete) mutation);
-          }
-          checkRow(mutation.getRow(), "doMiniBatchMutation");
-        } catch (NoSuchColumnFamilyException nscf) {
-          LOG.warn("No such column family in batch mutation", nscf);
-          batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
-              OperationStatusCode.BAD_FAMILY, nscf.getMessage());
-          lastIndexExclusive++;
-          continue;
-        } catch (FailedSanityCheckException fsce) {
-          LOG.warn("Batch Mutation did not pass sanity check", fsce);
-          batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
-              OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
-          lastIndexExclusive++;
-          continue;
-        } catch (WrongRegionException we) {
-          LOG.warn("Batch mutation had a row that does not belong to this region", we);
-          batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
-              OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
+        if (checkBatchOp(batchOp, lastIndexExclusive, familyMaps, now)) {
           lastIndexExclusive++;
           continue;
         }
-
-        // If we haven't got any rows in our batch, we should block to
-        // get the next one.
+        Mutation mutation = batchOp.getMutation(lastIndexExclusive);
+        // If we haven't got any rows in our batch, we should block to get the next one.
         RowLock rowLock = null;
         try {
           rowLock = getRowLock(mutation.getRow(), true);
         } catch (IOException ioe) {
-          LOG.warn("Failed getting lock in batch put, row="
-            + Bytes.toStringBinary(mutation.getRow()), ioe);
+          LOG.warn("Failed getting lock, row=" + Bytes.toStringBinary(mutation.getRow()), ioe);
         }
         if (rowLock == null) {
           // We failed to grab another lock
-          break; // stop acquiring more rows for this batch
+          break; // Stop acquiring more rows for this batch
         } else {
           acquiredRowLocks.add(rowLock);
         }
@@ -3080,9 +2991,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         lastIndexExclusive++;
         numReadyToWrite++;
 
-        if (isPutMutation) {
+        if (mutation instanceof Put) {
           // If Column Families stay consistent through out all of the
-          // individual puts then metrics can be reported as a mutliput across
+          // individual puts then metrics can be reported as a multiput across
           // column families in the first put.
           if (putsCfSet == null) {
             putsCfSet = mutation.getFamilyCellMap().keySet();
@@ -3100,23 +3011,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
       }
 
-      // we should record the timestamp only after we have acquired the rowLock,
+      // We've now grabbed as many mutations off the list as we can
+
+      // STEP 2. Update any LATEST_TIMESTAMP timestamps
+      // We should record the timestamp only after we have acquired the rowLock,
       // otherwise, newer puts/deletes are not guaranteed to have a newer timestamp
       now = EnvironmentEdgeManager.currentTime();
       byte[] byteNow = Bytes.toBytes(now);
 
       // Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily?
-      if (numReadyToWrite <= 0) return 0L;
-
-      // We've now grabbed as many mutations off the list as we can
+      if (numReadyToWrite <= 0) {
+        return 0L;
+      }
 
-      // ------------------------------------
-      // STEP 2. Update any LATEST_TIMESTAMP timestamps
-      // ----------------------------------
-      for (int i = firstIndex; !isInReplay && i < lastIndexExclusive; i++) {
+      for (int i = firstIndex; !replay && i < lastIndexExclusive; i++) {
         // skip invalid
         if (batchOp.retCodeDetails[i].getOperationStatusCode()
-            != OperationStatusCode.NOT_RUN) continue;
+            != OperationStatusCode.NOT_RUN) {
+          // lastIndexExclusive was incremented above.
+          continue;
+        }
 
         Mutation mutation = batchOp.getMutation(i);
         if (mutation instanceof Put) {
@@ -3133,16 +3047,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       locked = true;
 
       // calling the pre CP hook for batch mutation
-      if (!isInReplay && coprocessorHost != null) {
+      if (!replay && coprocessorHost != null) {
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
-        if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
+        if (coprocessorHost.preBatchMutate(miniBatchOp)) {
+          return 0L;
+        }
       }
 
-      // ------------------------------------
       // STEP 3. Build WAL edit
-      // ----------------------------------
       Durability durability = Durability.USE_DEFAULT;
       for (int i = firstIndex; i < lastIndexExclusive; i++) {
         // Skip puts that were determined to be invalid during preprocessing
@@ -3160,26 +3074,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           continue;
         }
 
-        long nonceGroup = batchOp.getNonceGroup(i), nonce = batchOp.getNonce(i);
+        long nonceGroup = batchOp.getNonceGroup(i);
+        long nonce = batchOp.getNonce(i);
         // In replay, the batch may contain multiple nonces. If so, write WALEdit for each.
         // Given how nonces are originally written, these should be contiguous.
         // They don't have to be, it will still work, just write more WALEdits than needed.
         if (nonceGroup != currentNonceGroup || nonce != currentNonce) {
-          if (walEdit.size() > 0) {
-            assert isInReplay;
-            if (!isInReplay) {
-              throw new IOException("Multiple nonces per batch and not in replay");
-            }
-            // txid should always increase, so having the one from the last call is ok.
-            // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-            walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
-              this.htableDescriptor.getTableName(), now, m.getClusterIds(),
-              currentNonceGroup, currentNonce, mvcc);
-            txid = this.wal.append(this.htableDescriptor,  this.getRegionInfo(),  walKey,
-              walEdit, true);
-            walEdit = new WALEdit(isInReplay);
-            walKey = null;
-          }
+          // Write what we have so far for nonces out to WAL
+          appendCurrentNonces(m, replay, walEdit, now, currentNonceGroup, currentNonce);
+          walEdit = new WALEdit(replay);
           currentNonceGroup = nonceGroup;
           currentNonce = nonce;
         }
@@ -3194,107 +3097,83 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         addFamilyMapToWALEdit(familyMaps[i], walEdit);
       }
 
-      // -------------------------
-      // STEP 4. Append the final edit to WAL. Do not sync wal.
-      // -------------------------
+      // STEP 4. Append the final edit to WAL and sync.
       Mutation mutation = batchOp.getMutation(firstIndex);
-      if (isInReplay) {
+      WALKey walKey = null;
+      if (replay) {
         // use wal key from the original
         walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
           this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
           mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
-        long replaySeqId = batchOp.getReplaySequenceId();
-        walKey.setOrigLogSeqNum(replaySeqId);
-      }
-      if (walEdit.size() > 0) {
-        if (!isInReplay) {
-        // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-        walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
+        walKey.setOrigLogSeqNum(batchOp.getReplaySequenceId());
+      }
+      // Not sure what is going on here when replay is going on... does the below append get
+      // called for replayed edits? Am afraid to change it without test.
+      if (!walEdit.isEmpty()) {
+        if (!replay) {
+          // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
+          walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
             this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
             mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
         }
-        txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+        // TODO: Use the doAppend methods below... complicated by the replay stuff above.
+        try {
+          long txid =
+            this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
+          if (txid != 0) sync(txid, durability);
+          writeEntry = walKey.getWriteEntry();
+        } catch (IOException ioe) {
+          if (walKey != null) mvcc.complete(walKey.getWriteEntry());
+          throw ioe;
+        }
       }
-      // ------------------------------------
-      // Acquire the latest mvcc number
-      // ----------------------------------
       if (walKey == null) {
-        // If this is a skip wal operation just get the read point from mvcc
-        walKey = this.appendEmptyEdit(this.wal);
-      }
-      if (!isInReplay) {
-        writeEntry = walKey.getWriteEntry();
-        mvccNum = writeEntry.getWriteNumber();
-      } else {
-        mvccNum = batchOp.getReplaySequenceId();
+        // If no walKey, then skipping WAL or some such. Being an mvcc transaction so sequenceid.
+        writeEntry = mvcc.begin();
       }
 
-      // ------------------------------------
       // STEP 5. Write back to memstore
-      // Write to memstore. It is ok to write to memstore
-      // first without syncing the WAL because we do not roll
-      // forward the memstore MVCC. The MVCC will be moved up when
-      // the complete operation is done. These changes are not yet
-      // visible to scanners till we update the MVCC. The MVCC is
-      // moved only when the sync is complete.
-      // ----------------------------------
       long addedSize = 0;
       for (int i = firstIndex; i < lastIndexExclusive; i++) {
-        if (batchOp.retCodeDetails[i].getOperationStatusCode()
-            != OperationStatusCode.NOT_RUN) {
+        if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
           continue;
         }
-        doRollBackMemstore = true; // If we have a failure, we need to clean what we wrote
-        addedSize += applyFamilyMapToMemstore(familyMaps[i], mvccNum, isInReplay);
+        addedSize += applyFamilyMapToMemstore(familyMaps[i], replay,
+            replay? batchOp.getReplaySequenceId(): writeEntry.getWriteNumber());
+      }
+
+      // STEP 6. Complete mvcc.
+      if (replay) {
+        this.mvcc.advanceTo(batchOp.getReplaySequenceId());
+      } else if (writeEntry != null/*Can be null if in replay mode*/) {
+        mvcc.completeAndWait(writeEntry);
+        writeEntry = null;
       }
 
-      // -------------------------------
-      // STEP 6. Release row locks, etc.
-      // -------------------------------
+      // STEP 7. Release row locks, etc.
       if (locked) {
         this.updatesLock.readLock().unlock();
         locked = false;
       }
       releaseRowLocks(acquiredRowLocks);
 
-      // -------------------------
-      // STEP 7. Sync wal.
-      // -------------------------
-      if (txid != 0) {
-        syncOrDefer(txid, durability);
-      }
-
-      doRollBackMemstore = false;
       // calling the post CP hook for batch mutation
-      if (!isInReplay && coprocessorHost != null) {
+      if (!replay && coprocessorHost != null) {
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         coprocessorHost.postBatchMutate(miniBatchOp);
       }
 
-      // ------------------------------------------------------------------
-      // STEP 8. Advance mvcc. This will make this put visible to scanners and getters.
-      // ------------------------------------------------------------------
-      if (writeEntry != null) {
-        mvcc.completeAndWait(writeEntry);
-        writeEntry = null;
-      } else if (isInReplay) {
-        // ensure that the sequence id of the region is at least as big as orig log seq id
-        mvcc.advanceTo(mvccNum);
-      }
-
       for (int i = firstIndex; i < lastIndexExclusive; i ++) {
         if (batchOp.retCodeDetails[i] == OperationStatus.NOT_RUN) {
           batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
         }
       }
 
-      // ------------------------------------
-      // STEP 9. Run coprocessor post hooks. This should be done after the wal is
+      // STEP 8. Run coprocessor post hooks. This should be done after the wal is
       // synced so that the coprocessor contract is adhered to.
-      // ------------------------------------
-      if (!isInReplay && coprocessorHost != null) {
+      if (!replay && coprocessorHost != null) {
         for (int i = firstIndex; i < lastIndexExclusive; i++) {
           // only for successful puts
           if (batchOp.retCodeDetails[i].getOperationStatusCode()
@@ -3313,18 +3192,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       success = true;
       return addedSize;
     } finally {
-      // if the wal sync was unsuccessful, remove keys from memstore
-      if (doRollBackMemstore) {
-        for (int j = 0; j < familyMaps.length; j++) {
-          for(List<Cell> cells:familyMaps[j].values()) {
-            rollbackMemstore(cells);
-          }
-        }
-        if (writeEntry != null) mvcc.complete(writeEntry);
-      } else if (writeEntry != null) {
-        mvcc.completeAndWait(writeEntry);
-      }
-
+      // Call complete rather than completeAndWait because we probably had error if walKey != null
+      if (writeEntry != null) mvcc.complete(writeEntry);
       if (locked) {
         this.updatesLock.readLock().unlock();
       }
@@ -3369,6 +3238,88 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void appendCurrentNonces(final Mutation mutation, final boolean replay,
+      final WALEdit walEdit, final long now, final long currentNonceGroup, final long currentNonce)
+  throws IOException {
+    if (walEdit.isEmpty()) return;
+    if (!replay) throw new IOException("Multiple nonces per batch and not in replay");
+    WALKey walKey = new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
+        this.htableDescriptor.getTableName(), now, mutation.getClusterIds(),
+        currentNonceGroup, currentNonce, mvcc);
+    this.wal.append(this.htableDescriptor,  this.getRegionInfo(), walKey, walEdit, true);
+    // Complete the mvcc transaction started down in append else it will block others
+    this.mvcc.complete(walKey.getWriteEntry());
+  }
+
+  private boolean checkBatchOp(BatchOperation<?> batchOp, final int lastIndexExclusive,
+      final Map<byte[], List<Cell>>[] familyMaps, final long now)
+  throws IOException {
+    boolean skip = false;
+    // Skip anything that "ran" already
+    if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
+        != OperationStatusCode.NOT_RUN) {
+      return true;
+    }
+    Mutation mutation = batchOp.getMutation(lastIndexExclusive);
+    Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
+    // store the family map reference to allow for mutations
+    familyMaps[lastIndexExclusive] = familyMap;
+
+    try {
+      if (mutation instanceof Put) {
+        // Check the families in the put. If bad, skip this one.
+        if (batchOp.isInReplay()) {
+          removeNonExistentColumnFamilyForReplay(familyMap);
+        } else {
+          checkFamilies(familyMap.keySet());
+        }
+        checkTimestamps(mutation.getFamilyCellMap(), now);
+      } else {
+        prepareDelete((Delete)mutation);
+      }
+      checkRow(mutation.getRow(), "doMiniBatchMutation");
+    } catch (NoSuchColumnFamilyException nscf) {
+      LOG.warn("No such column family in batch mutation", nscf);
+      batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
+          OperationStatusCode.BAD_FAMILY, nscf.getMessage());
+      skip = true;
+    } catch (FailedSanityCheckException fsce) {
+      LOG.warn("Batch Mutation did not pass sanity check", fsce);
+      batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
+          OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
+      skip = true;
+    } catch (WrongRegionException we) {
+      LOG.warn("Batch mutation had a row that does not belong to this region", we);
+      batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
+          OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
+      skip = true;
+    }
+    return skip;
+  }
+
+  /**
+   * During replay, there could exist column families which are removed between region server
+   * failure and replay
+   */
+  private void removeNonExistentColumnFamilyForReplay(final Map<byte[], List<Cell>> familyMap) {
+    List<byte[]> nonExistentList = null;
+    for (byte[] family : familyMap.keySet()) {
+      if (!this.htableDescriptor.hasFamily(family)) {
+        if (nonExistentList == null) {
+          nonExistentList = new ArrayList<byte[]>();
+        }
+        nonExistentList.add(family);
+      }
+    }
+    if (nonExistentList != null) {
+      for (byte[] family : nonExistentList) {
+        // Perhaps schema was changed between crash and replay
+        LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
+        familyMap.remove(family);
+      }
+    }
+  }
+
   /**
    * Returns effective durability from the passed durability and
    * the table descriptor.
@@ -3377,93 +3328,82 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return d == Durability.USE_DEFAULT ? this.durability : d;
   }
 
-  //TODO, Think that gets/puts and deletes should be refactored a bit so that
-  //the getting of the lock happens before, so that you would just pass it into
-  //the methods. So in the case of checkAndMutate you could just do lockRow,
-  //get, put, unlockRow or something
-
   @Override
   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
+      CompareOp compareOp, ByteArrayComparable comparator, Mutation mutation,
       boolean writeToWAL)
   throws IOException{
+    checkMutationType(mutation, row);
+    return doCheckAndRowMutate(row, family, qualifier, compareOp, comparator, null,
+      mutation, writeToWAL);
+  }
+
+  @Override
+  public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
+      CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
+      boolean writeToWAL)
+  throws IOException {
+    return doCheckAndRowMutate(row, family, qualifier, compareOp, comparator, rm, null,
+      writeToWAL);
+  }
+
+  /**
+   * checkAndMutate and checkAndRowMutate are 90% the same. Rather than copy/paste, below has
+   * switches in the few places where there is deviation.
+   */
+  private boolean doCheckAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
+      CompareOp compareOp, ByteArrayComparable comparator, RowMutations rowMutations,
+      Mutation mutation, boolean writeToWAL)
+  throws IOException {
+    // Could do the below checks but seems wacky with two callers only. Just comment out for now.
+    // One caller passes a Mutation, the other passes RowMutation. Presume all good so we don't
+    // need these commented out checks.
+    // if (rowMutations == null && mutation == null) throw new DoNotRetryIOException("Both null");
+    // if (rowMutations != null && mutation != null) throw new DoNotRetryIOException("Both set");
     checkReadOnly();
-    //TODO, add check for value length or maybe even better move this to the
-    //client if this becomes a global setting
+    // TODO, add check for value length also move this check to the client
     checkResources();
-    boolean isPut = w instanceof Put;
-    if (!isPut && !(w instanceof Delete))
-      throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
-          "be Put or Delete");
-    if (!Bytes.equals(row, w.getRow())) {
-      throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
-          "getRow must match the passed row");
-    }
-
     startRegionOperation();
     try {
       Get get = new Get(row);
       checkFamily(family);
       get.addColumn(family, qualifier);
-
       // Lock row - note that doBatchMutate will relock this row if called
       RowLock rowLock = getRowLock(get.getRow());
-      // wait for all previous transactions to complete (with lock held)
-      mvcc.await();
       try {
-        if (this.getCoprocessorHost() != null) {
+        if (mutation != null && this.getCoprocessorHost() != null) {
+          // Call coprocessor.
           Boolean processed = null;
-          if (w instanceof Put) {
+          if (mutation instanceof Put) {
             processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
-                qualifier, compareOp, comparator, (Put) w);
-          } else if (w instanceof Delete) {
+                qualifier, compareOp, comparator, (Put)mutation);
+          } else if (mutation instanceof Delete) {
             processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
-                qualifier, compareOp, comparator, (Delete) w);
+                qualifier, compareOp, comparator, (Delete)mutation);
           }
           if (processed != null) {
             return processed;
           }
         }
+        // NOTE: We used to wait here until mvcc caught up:  mvcc.await();
+        // Supposition is that now all changes are done under row locks, then when we go to read,
+        // we'll get the latest on this row.
         List<Cell> result = get(get, false);
-
-        boolean valueIsNull = comparator.getValue() == null ||
-          comparator.getValue().length == 0;
+        boolean valueIsNull = comparator.getValue() == null || comparator.getValue().length == 0;
         boolean matches = false;
         long cellTs = 0;
         if (result.size() == 0 && valueIsNull) {
           matches = true;
-        } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
-            valueIsNull) {
+        } else if (result.size() > 0 && result.get(0).getValueLength() == 0 && valueIsNull) {
           matches = true;
           cellTs = result.get(0).getTimestamp();
         } else if (result.size() == 1 && !valueIsNull) {
           Cell kv = result.get(0);
           cellTs = kv.getTimestamp();
           int compareResult = CellComparator.compareValue(kv, comparator);
-          switch (compareOp) {
-          case LESS:
-            matches = compareResult < 0;
-            break;
-          case LESS_OR_EQUAL:
-            matches = compareResult <= 0;
-            break;
-          case EQUAL:
-            matches = compareResult == 0;
-            break;
-          case NOT_EQUAL:
-            matches = compareResult != 0;
-            break;
-          case GREATER_OR_EQUAL:
-            matches = compareResult >= 0;
-            break;
-          case GREATER:
-            matches = compareResult > 0;
-            break;
-          default:
-            throw new RuntimeException("Unknown Compare op " + compareOp.name());
-          }
+          matches = matches(compareOp, compareResult);
         }
-        //If matches put the new put or delete the new delete
+        // If matches put the new put or delete the new delete
         if (matches) {
           // We have acquired the row lock already. If the system clock is NOT monotonically
           // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
@@ -3472,16 +3412,27 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           long now = EnvironmentEdgeManager.currentTime();
           long ts = Math.max(now, cellTs); // ensure write is not eclipsed
           byte[] byteTs = Bytes.toBytes(ts);
-
-          if (w instanceof Put) {
-            updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
+          if (mutation != null) {
+            if (mutation instanceof Put) {
+              updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs);
+            }
+            // And else 'delete' is not needed since it already does a second get, and sets the
+            // timestamp from get (see prepareDeleteTimestamps).
+          } else {
+            for (Mutation m: rowMutations.getMutations()) {
+              if (m instanceof Put) {
+                updateCellTimestamps(m.getFamilyCellMap().values(), byteTs);
+              }
+            }
+            // And else 'delete' is not needed since it already does a second get, and sets the
+            // timestamp from get (see prepareDeleteTimestamps).
+          }
+          // All edits for the given row (across all column families) must happen atomically.
+          if (mutation != null) {
+            doBatchMutate(mutation);
+          } else {
+            mutateRow(rowMutations);
           }
-          // else delete is not needed since it already does a second get, and sets the timestamp
-          // from get (see prepareDeleteTimestamps).
-
-          // All edits for the given row (across all column families) must
-          // happen atomically.
-          doBatchMutate(w);
           this.checkAndMutateChecksPassed.increment();
           return true;
         }
@@ -3495,113 +3446,54 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  //TODO, Think that gets/puts and deletes should be refactored a bit so that
-  //the getting of the lock happens before, so that you would just pass it into
-  //the methods. So in the case of checkAndMutate you could just do lockRow,
-  //get, put, unlockRow or something
+  private void checkMutationType(final Mutation mutation, final byte [] row)
+  throws DoNotRetryIOException {
+    boolean isPut = mutation instanceof Put;
+    if (!isPut && !(mutation instanceof Delete)) {
+      throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must be Put or Delete");
+    }
+    if (!Bytes.equals(row, mutation.getRow())) {
+      throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match");
+    }
+  }
 
-  @Override
-  public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
-      CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
-      boolean writeToWAL) throws IOException {
-    checkReadOnly();
-    //TODO, add check for value length or maybe even better move this to the
-    //client if this becomes a global setting
-    checkResources();
+  private boolean matches(final CompareOp compareOp, final int compareResult) {
+    boolean matches = false;
+    switch (compareOp) {
+      case LESS:
+        matches = compareResult < 0;
+        break;
+      case LESS_OR_EQUAL:
+        matches = compareResult <= 0;
+        break;
+      case EQUAL:
+        matches = compareResult == 0;
+        break;
+      case NOT_EQUAL:
+        matches = compareResult != 0;
+        break;
+      case GREATER_OR_EQUAL:
+        matches = compareResult >= 0;
+        break;
+      case GREATER:
+        matches = compareResult > 0;
+        break;
+      default:
+        throw new RuntimeException("Unknown Compare op " + compareOp.name());
+    }
+    return matches;
+  }
 
-    startRegionOperation();
-    try {
-      Get get = new Get(row);
-      checkFamily(family);
-      get.addColumn(family, qualifier);
 
-      // Lock row - note that doBatchMutate will relock this row if called
-      RowLock rowLock = getRowLock(get.getRow());
-      // wait for all previous transactions to complete (with lock held)
-      mvcc.await();
-      try {
-        List<Cell> result = get(get, false);
-
-        boolean valueIsNull = comparator.getValue() == null ||
-            comparator.getValue().length == 0;
-        boolean matches = false;
-        long cellTs = 0;
-        if (result.size() == 0 && valueIsNull) {
-          matches = true;
-        } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
-            valueIsNull) {
-          matches = true;
-          cellTs = result.get(0).getTimestamp();
-        } else if (result.size() == 1 && !valueIsNull) {
-          Cell kv = result.get(0);
-          cellTs = kv.getTimestamp();
-          int compareResult = CellComparator.compareValue(kv, comparator);
-          switch (compareOp) {
-          case LESS:
-            matches = compareResult < 0;
-            break;
-          case LESS_OR_EQUAL:
-            matches = compareResult <= 0;
-            break;
-          case EQUAL:
-            matches = compareResult == 0;
-            break;
-          case NOT_EQUAL:
-            matches = compareResult != 0;
-            break;
-          case GREATER_OR_EQUAL:
-            matches = compareResult >= 0;
-            break;
-          case GREATER:
-            matches = compareResult > 0;
-            break;
-          default:
-            throw new RuntimeException("Unknown Compare op " + compareOp.name());
-          }
-        }
-        //If matches put the new put or delete the new delete
-        if (matches) {
-          // We have acquired the row lock already. If the system clock is NOT monotonically
-          // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
-          // larger timestamp than what was observed via Get. doBatchMutate already does this, but
-          // there is no way to pass the cellTs. See HBASE-14054.
-          long now = EnvironmentEdgeManager.currentTime();
-          long ts = Math.max(now, cellTs); // ensure write is not eclipsed
-          byte[] byteTs = Bytes.toBytes(ts);
-
-          for (Mutation w : rm.getMutations()) {
-            if (w instanceof Put) {
-              updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
-            }
-            // else delete is not needed since it already does a second get, and sets the timestamp
-            // from get (see prepareDeleteTimestamps).
-          }
-
-          // All edits for the given row (across all column families) must
-          // happen atomically.
-          mutateRow(rm);
-          this.checkAndMutateChecksPassed.increment();
-          return true;
-        }
-        this.checkAndMutateChecksFailed.increment();
-        return false;
-      } finally {
-        rowLock.release();
-      }
-    } finally {
-      closeRegionOperation();
-    }
-  }
-
-  private void doBatchMutate(Mutation mutation) throws IOException {
-    // Currently this is only called for puts and deletes, so no nonces.
-    OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
-    if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
-      throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
-    } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
-      throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
-    }
-  }
+  private void doBatchMutate(Mutation mutation) throws IOException {
+    // Currently this is only called for puts and deletes, so no nonces.
+    OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
+    if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
+      throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
+    } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
+      throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
+    }
+  }
 
   /**
    * Complete taking the snapshot on the region. Writes the region info and adds references to the
@@ -3663,40 +3555,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
     // Check if we have any work to do and early out otherwise
     // Update these checks as more logic is added here
-
     if (m.getTTL() == Long.MAX_VALUE) {
       return;
     }
 
     // From this point we know we have some work to do
-
     for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
       List<Cell> cells = e.getValue();
       assert cells instanceof RandomAccess;
       int listSize = cells.size();
       for (int i = 0; i < listSize; i++) {
         Cell cell = cells.get(i);
-        List<Tag> newTags = new ArrayList<Tag>();
-        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell);
-
-        // Carry forward existing tags
-
-        while (tagIterator.hasNext()) {
-
-          // Add any filters or tag specific rewrites here
-
-          newTags.add(tagIterator.next());
-        }
-
-        // Cell TTL handling
-
-        // Check again if we need to add a cell TTL because early out logic
-        // above may change when there are more tag based features in core.
-        if (m.getTTL() != Long.MAX_VALUE) {
-          // Add a cell TTL tag
-          newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL())));
-        }
-
+        List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
+        newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
         // Rewrite the cell with the updated set of tags
         cells.set(i, new TagRewriteCell(cell, TagUtil.fromList(newTags)));
       }
@@ -3772,49 +3643,64 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * should already have locked updatesLock.readLock(). This also does
    * <b>not</b> check the families for validity.
    *
-   * @param familyMap Map of kvs per family
-   * @param mvccNum The MVCC for this transaction.
-   * @param isInReplay true when adding replayed KVs into memstore
-   * @return the additional memory usage of the memstore caused by the
-   * new entries.
+   * @param familyMap Map of Cells by family
+   * @return the additional memory usage of the memstore caused by the new entries.
    */
-  private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap,
-    long mvccNum, boolean isInReplay) throws IOException {
+  private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap, boolean replay,
+      long sequenceId)
+  throws IOException {
     long size = 0;
-
     for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
       byte[] family = e.getKey();
       List<Cell> cells = e.getValue();
       assert cells instanceof RandomAccess;
-      Store store = getStore(family);
-      int listSize = cells.size();
-      for (int i=0; i < listSize; i++) {
+      size += applyToMemstore(getStore(family), cells, false, replay, sequenceId);
+    }
+    return size;
+  }
+
+  /**
+   * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
+   *  set; when set we will run operations that make sense in the increment/append scenario but
+   *  that do not make sense otherwise.
+   * @return Memstore change in size on insert of these Cells.
+   * @see #applyToMemstore(Store, Cell, long)
+   */
+  private long applyToMemstore(final Store store, final List<Cell> cells,
+      final boolean delta, boolean replay, long sequenceId)
+  throws IOException {
+    // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
+    long size = 0;
+    boolean upsert = delta && store.getFamily().getMaxVersions() == 1;
+    int count = cells.size();
+    if (upsert) {
+      size += store.upsert(cells, getSmallestReadPoint());
+    } else {
+      for (int i = 0; i < count; i++) {
         Cell cell = cells.get(i);
-        if (cell.getSequenceId() == 0 || isInReplay) {
-          CellUtil.setSequenceId(cell, mvccNum);
+        // TODO: This looks wrong.. checking for sequenceid of zero is expensive!!!!! St.Ack
+        // When is it zero anyways? When replay? Then just rely on that flag.
+        if (cell.getSequenceId() == 0 || replay) {
+          CellUtil.setSequenceId(cell, sequenceId);
         }
         size += store.add(cell);
       }
     }
-
-     return size;
-   }
+    return size;
+  }
 
   /**
-   * Remove all the keys listed in the map from the memstore. This method is
-   * called when a Put/Delete has updated memstore but subsequently fails to update
-   * the wal. This method is then invoked to rollback the memstore.
+   * @return Memstore change in size on insert of these Cells.
+   * @see #applyToMemstore(Store, List, boolean, boolean, long)
    */
-  private void rollbackMemstore(List<Cell> memstoreCells) {
-    int kvsRolledback = 0;
-
-    for (Cell cell : memstoreCells) {
-      byte[] family = CellUtil.cloneFamily(cell);
-      Store store = getStore(family);
-      store.rollback(cell);
-      kvsRolledback++;
+  private long applyToMemstore(final Store store, final Cell cell, long sequenceId)
+  throws IOException {
+    // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
+    if (store == null) {
+      checkFamily(CellUtil.cloneFamily(cell));
+      // Unreachable because checkFamily will throw exception
     }
-    LOG.debug("rollbackMemstore rolled back " + kvsRolledback);
+    return store.add(cell);
   }
 
   @Override
@@ -3824,30 +3710,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  /**
-   * During replay, there could exist column families which are removed between region server
-   * failure and replay
-   */
-  private void removeNonExistentColumnFamilyForReplay(
-      final Map<byte[], List<Cell>> familyMap) {
-    List<byte[]> nonExistentList = null;
-    for (byte[] family : familyMap.keySet()) {
-      if (!this.htableDescriptor.hasFamily(family)) {
-        if (nonExistentList == null) {
-          nonExistentList = new ArrayList<byte[]>();
-        }
-        nonExistentList.add(family);
-      }
-    }
-    if (nonExistentList != null) {
-      for (byte[] family : nonExistentList) {
-        // Perhaps schema was changed between crash and replay
-        LOG.info("No family for " + Bytes.toString(family) + " omit from reply.");
-        familyMap.remove(family);
-      }
-    }
-  }
-
   @Override
   public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
       throws FailedSanityCheckException {
@@ -5490,12 +5352,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       return true;
     } finally {
       if (wal != null && !storeFiles.isEmpty()) {
-        // write a bulk load event when not all hfiles are loaded
+        // @rite a bulk load event when not all hfiles are loaded
         try {
           WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
               this.getRegionInfo().getTable(),
               ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
-          WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
+          WALUtil.writeBulkLoadMarkerAndSync(this.wal, getTableDesc(), getRegionInfo(),
               loadDescriptor, mvcc);
         } catch (IOException ioe) {
           if (this.rsServices != null) {
@@ -5593,7 +5455,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // getSmallestReadPoint, before scannerReadPoints is updated.
       IsolationLevel isolationLevel = scan.getIsolationLevel();
       synchronized(scannerReadPoints) {
-        this.readPt = getReadpoint(isolationLevel);
+        this.readPt = getReadPoint(isolationLevel);
         scannerReadPoints.put(this, this.readPt);
       }
 
@@ -5758,7 +5620,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
       // As the data is obtained from two independent heaps, we need to
       // ensure that result list is sorted, because Result relies on that.
-      Collections.sort(results, comparator);
+      sort(results, comparator);
       return moreValues;
     }
 
@@ -6876,7 +6738,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
       long nonceGroup, long nonce) throws IOException {
-
     for (byte[] row : processor.getRowsToLock()) {
       checkRow(row, "processRowsWithLocks");
     }
@@ -6884,23 +6745,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       checkReadOnly();
     }
     checkResources();
-
     startRegionOperation();
     WALEdit walEdit = new WALEdit();
 
-    // 1. Run pre-process hook
-    try {
-      processor.preProcess(this, walEdit);
-    } catch (IOException e) {
-      closeRegionOperation();
-      throw e;
-    }
+    // STEP 1. Run pre-process hook
+    preProcess(processor, walEdit);
     // Short circuit the read only case
     if (processor.readOnly()) {
       try {
         long now = EnvironmentEdgeManager.currentTime();
-        doProcessRowWithTimeout(
-            processor, now, this, null, null, timeout);
+        doProcessRowWithTimeout(processor, now, this, null, null, timeout);
         processor.postProcess(this, walEdit, true);
       } finally {
         closeRegionOperation();
@@ -6908,118 +6762,81 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       return;
     }
 
-    MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
     boolean locked;
-    boolean walSyncSuccessful = false;
     List<RowLock> acquiredRowLocks;
     long addedSize = 0;
     List<Mutation> mutations = new ArrayList<Mutation>();
     Collection<byte[]> rowsToLock = processor.getRowsToLock();
-    long mvccNum = 0;
-    WALKey walKey = null;
+    // This is assigned by mvcc either explicity in the below or in the guts of the WAL append
+    // when it assigns the edit a sequencedid (A.K.A the mvcc write number).
+    WriteEntry writeEntry = null;
     try {
-      // 2. Acquire the row lock(s)
+      // STEP 2. Acquire the row lock(s)
       acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
       for (byte[] row : rowsToLock) {
         // Attempt to lock all involved rows, throw if any lock times out
         // use a writer lock for mixed reads and writes
         acquiredRowLocks.add(getRowLock(row));
       }
-      // 3. Region lock
+      // STEP 3. Region lock
       lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size());
       locked = true;
-
+      boolean success = false;
       long now = EnvironmentEdgeManager.currentTime();
       try {
-        // 4. Let the processor scan the rows, generate mutations and add
-        //    waledits
-        doProcessRowWithTimeout(
-            processor, now, this, mutations, walEdit, timeout);
-
+        // STEP 4. Let the processor scan the rows, generate mutations and add waledits
+        doProcessRowWithTimeout(processor, now, this, mutations, walEdit, timeout);
         if (!mutations.isEmpty()) {
-
-          // 5. Call the preBatchMutate hook
+          // STEP 5. Call the preBatchMutate hook
           processor.preBatchMutate(this, walEdit);
 
-          long txid = 0;
-          // 6. Append no sync
+          // STEP 6. Append and sync if walEdit has data to write out.
           if (!walEdit.isEmpty()) {
-            // we use HLogKey here instead of WALKey directly to support legacy coprocessors.
-            walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
-              this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
-              processor.getClusterIds(), nonceGroup, nonce, mvcc);
-            txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
-                walKey, walEdit, false);
-          }
-          if(walKey == null){
-            // since we use wal sequence Id as mvcc, for SKIP_WAL changes we need a "faked" WALEdit
-            // to get a sequence id assigned which is done by FSWALEntry#stampRegionSequenceId
-            walKey = this.appendEmptyEdit(this.wal);
+            writeEntry = doWALAppend(walEdit, getEffectiveDurability(processor.useDurability()),
+                processor.getClusterIds(), now, nonceGroup, nonce);
+          } else {
+            // We are here if WAL is being skipped.
+            writeEntry = this.mvcc.begin();
           }
 
-          // 7. Start mvcc transaction
-          writeEntry = walKey.getWriteEntry();
-          mvccNum = walKey.getSequenceId();
-
-
-
-          // 8. Apply to memstore
+          // STEP 7. Apply to memstore
+          long sequenceId = writeEntry.getWriteNumber();
           for (Mutation m : mutations) {
-            // Handle any tag based cell features
+            // Handle any tag based cell features.
+            // TODO: Do we need to call rewriteCellTags down in applyToMemstore()? Why not before
+            // so tags go into WAL?
             rewriteCellTags(m.getFamilyCellMap(), m);
-
             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
               Cell cell = cellScanner.current();
-              CellUtil.setSequenceId(cell, mvccNum);
-              Store store = getStore(cell);
-              if (store == null) {
-                checkFamily(CellUtil.cloneFamily(cell));
-                // unreachable
+              if (walEdit.isEmpty()) {
+                // If walEdit is empty, we put nothing in WAL. WAL stamps Cells with sequence id.
+                // If no WAL, need to stamp it here.
+                CellUtil.setSequenceId(cell, sequenceId);
               }
-              addedSize += store.add(cell);
+              Store store = getStore(cell);
+              addedSize += applyToMemstore(store, cell, sequenceId);
             }
           }
+          // STEP 8. Complete mvcc.
+          mvcc.completeAndWait(writeEntry);
+          writeEntry = null;
 
-          // 9. Release region lock
+          // STEP 9. Release region lock
           if (locked) {
             this.updatesLock.readLock().unlock();
             locked = false;
           }
 
-          // 10. Release row lock(s)
+          // STEP 10. Release row lock(s)
           releaseRowLocks(acquiredRowLocks);
 
-          // 11. Sync edit log
-          if (txid != 0) {
-            syncOrDefer(txid, getEffectiveDurability(processor.useDurability()));
-          }
-          walSyncSuccessful = true;
-          // 12. call postBatchMutate hook
+          // STEP 11. call postBatchMutate hook
           processor.postBatchMutate(this);
         }
+        success = true;
       } finally {
-        // TODO: Make this method look like all other methods that are doing append/sync and
-        // memstore rollback such as append and doMiniBatchMutation. Currently it is a little
-        // different. Make them all share same code!
-        if (!mutations.isEmpty() && !walSyncSuccessful) {
-          LOG.warn("Wal sync failed. Roll back " + mutations.size() +
-              " memstore keyvalues for row(s):" + StringUtils.byteToHexString(
-              processor.getRowsToLock().iterator().next()) + "...");
-          for (Mutation m : mutations) {
-            for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
-              Cell cell = cellScanner.current();
-              getStore(cell).rollback(cell);
-            }
-          }
-          if (writeEntry != null) {
-            mvcc.complete(writeEntry);
-            writeEntry = null;
-          }
-        }
-        // 13. Roll mvcc forward
-        if (writeEntry != null) {
-          mvcc.completeAndWait(writeEntry);
-        }
+        // Call complete rather than completeAndWait because we probably had error if walKey != null
+        if (writeEntry != null) mvcc.complete(writeEntry);
         if (locked) {
           this.updatesLock.readLock().unlock();
         }
@@ -7027,18 +6844,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         releaseRowLocks(acquiredRowLocks);
       }
 
-      // 14. Run post-process hook
-      processor.postProcess(this, walEdit, walSyncSuccessful);
-
+      // 12. Run post-process hook
+      processor.postProcess(this, walEdit, success);
     } finally {
       closeRegionOperation();
-      if (!mutations.isEmpty() &&
-          isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
+      if (!mutations.isEmpty() && isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
         requestFlush();
       }
     }
   }
 
+  private void preProcess(final RowProcessor<?,?> processor, final WALEdit walEdit)
+  throws IOException {
+    try {
+      processor.preProcess(this, walEdit);
+    } catch (IOException e) {
+      closeRegionOperation();
+      throw e;
+    }
+  }
+
   private void doProcessRowWithTimeout(final RowProcessor<?,?> processor,
                                        final long now,
                                        final HRegion region,
@@ -7089,500 +6914,400 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  /**
-   * @return The passed-in {@code tags} but with the tags from {@code cell} added.
-   */
-  private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
-    if (cell.getTagsLength() <= 0) return tags;
-    List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags;
-    Iterator<Tag> i = CellUtil.tagsIterator(cell);
-    while (i.hasNext()) newTags.add(i.next());
-    return newTags;
+  public Result append(Append append) throws IOException {
+    return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
   }
 
-  /**
-   * Run a Get against passed in <code>store</code> on passed <code>row</code>, etc.
-   * @return Get result.
-   */
-  private List<Cell> doGet(final Store store, final byte [] row,
-      final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
-  throws IOException {
-    // Sort the cells so that they match the order that they
-    // appear in the Get results. Otherwise, we won't be able to
-    // find the existing values if the cells are not specified
-    // in order by the client since cells are in an array list.
-    Collections.sort(family.getValue(), store.getComparator());
-    // Get previous values for all columns in this family
-    Get get = new Get(row);
-    for (Cell cell : family.getValue()) {
-      get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell));
-    }
-    if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax());
-    return get(get, false);
+  @Override
+  public Result append(Append mutation, long nonceGroup, long nonce) throws IOException {
+    return doDelta(Operation.APPEND, mutation, nonceGroup, nonce, mutation.isReturnResults());
   }
 
-  public Result append(Append append) throws IOException {
-    return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
+  public Result increment(Increment increment) throws IOException {
+    return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
   }
 
-  // TODO: There's a lot of boiler plate code identical to increment.
-  // We should refactor append and increment as local get-mutate-put
-  // transactions, so all stores only go through one code path for puts.
-
   @Override
-  public Result append(Append mutate, long nonceGroup, long nonce) throws IOException {
-    Operation op = Operation.APPEND;
-    byte[] row = mutate.getRow();
-    checkRow(row, op.toString());
-    checkFamilies(mutate.getFamilyCellMap().keySet());
-    boolean flush = false;
-    Durability durability = getEffectiveDurability(mutate.getDurability());
-    boolean writeToWAL = durability != Durability.SKIP_WAL;
-    WALEdit walEdits = null;
-    List<Cell> allKVs = new ArrayList<Cell>(mutate.size());
-    Map<Store, List<Cell>> tempMemstore = new HashMap<Store, List<Cell>>();
-    long size = 0;
-    long txid = 0;
+  public Result increment(Increment mutation, long nonceGroup, long nonce)
+  throws IOException {
+    return doDelta(Operation.INCREMENT, mutation, nonceGroup, nonce, mutation.isReturnResults());
+  }
+
+  /**
+   * Add "deltas" to Cells. Deltas are increments or appends. Switch on <code>op</code>.
+   *
+   * <p>If increment, add deltas to current values or if an append, then
+   * append the deltas to the current Cell values.
+   *
+   * <p>Append and Increment code paths are mostly the same. They differ in just a few pl

<TRUNCATED>

[12/32] hbase git commit: HBASE-15197 Expose filtered read requests metric to metrics framework and Web UI (Eungsop Yoo)

Posted by sy...@apache.org.
HBASE-15197 Expose filtered read requests metric to metrics framework and Web UI (Eungsop Yoo)


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

Branch: refs/heads/hbase-12439
Commit: 8f20bc748de60f8da3ab7f66c6a198515d53de35
Parents: b80325f
Author: chenheng <ch...@apache.org>
Authored: Fri Feb 5 10:57:14 2016 +0800
Committer: chenheng <ch...@apache.org>
Committed: Fri Feb 5 10:57:14 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/RegionLoad.java     |   7 +
 .../org/apache/hadoop/hbase/ServerLoad.java     |   8 +
 .../regionserver/MetricsRegionServerSource.java |   3 +
 .../MetricsRegionServerWrapper.java             |   5 +
 .../regionserver/MetricsRegionWrapper.java      |   5 +
 .../MetricsRegionServerSourceImpl.java          |   2 +
 .../regionserver/MetricsRegionSourceImpl.java   |   4 +
 .../TestMetricsRegionSourceImpl.java            |   5 +
 .../protobuf/generated/ClusterStatusProtos.java | 191 ++++++++--
 .../src/main/protobuf/ClusterStatus.proto       |   3 +
 .../tmpl/master/RegionServerListTmpl.jamon      |   2 +
 .../tmpl/regionserver/RegionListTmpl.jamon      |   2 +
 .../tmpl/regionserver/ServerMetricsTmpl.jamon   |   2 +
 .../hadoop/hbase/regionserver/HRegion.java      |  13 +-
 .../hbase/regionserver/HRegionServer.java       |   1 +
 .../MetricsRegionServerWrapperImpl.java         |  11 +-
 .../regionserver/MetricsRegionWrapperImpl.java  |   5 +
 .../hadoop/hbase/regionserver/Region.java       |   3 +
 .../hadoop/hbase/TestRegionServerMetrics.java   | 379 +++++++++++++++++++
 .../org/apache/hadoop/hbase/TestServerLoad.java |   4 +
 .../MetricsRegionServerWrapperStub.java         |   5 +
 .../regionserver/MetricsRegionWrapperStub.java  |   5 +
 .../hbase/regionserver/TestMetricsRegion.java   |   8 +
 .../regionserver/TestMetricsRegionServer.java   |   1 +
 24 files changed, 638 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
index a6e846e..5bf2ec7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
@@ -106,6 +106,13 @@ public class RegionLoad {
   }
 
   /**
+   * @return the number of filtered read requests made to region
+   */
+  public long getFilteredReadRequestsCount() {
+    return regionLoadPB.getFilteredReadRequestsCount();
+  }
+
+  /**
    * @return the number of write requests made to region
    */
   public long getWriteRequestsCount() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 60fae85..1ddcc20 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -49,6 +49,7 @@ public class ServerLoad {
   private int memstoreSizeMB = 0;
   private int storefileIndexSizeMB = 0;
   private long readRequestsCount = 0;
+  private long filteredReadRequestsCount = 0;
   private long writeRequestsCount = 0;
   private int rootIndexSizeKB = 0;
   private int totalStaticIndexSizeKB = 0;
@@ -66,6 +67,7 @@ public class ServerLoad {
       memstoreSizeMB += rl.getMemstoreSizeMB();
       storefileIndexSizeMB += rl.getStorefileIndexSizeMB();
       readRequestsCount += rl.getReadRequestsCount();
+      filteredReadRequestsCount += rl.getFilteredReadRequestsCount();
       writeRequestsCount += rl.getWriteRequestsCount();
       rootIndexSizeKB += rl.getRootIndexSizeKB();
       totalStaticIndexSizeKB += rl.getTotalStaticIndexSizeKB();
@@ -145,6 +147,10 @@ public class ServerLoad {
     return readRequestsCount;
   }
 
+  public long getFilteredReadRequestsCount() {
+    return filteredReadRequestsCount;
+  }
+
   public long getWriteRequestsCount() {
     return writeRequestsCount;
   }
@@ -297,6 +303,8 @@ public class ServerLoad {
         Strings.appendKeyValue(sb, "storefileIndexSizeMB",
           Integer.valueOf(this.storefileIndexSizeMB));
     sb = Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
+    sb = Strings.appendKeyValue(sb, "filteredReadRequestsCount",
+      Long.valueOf(this.filteredReadRequestsCount));
     sb = Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
     sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
     sb =

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index ee0217a..e4df1c0 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -162,6 +162,9 @@ public interface MetricsRegionServerSource extends BaseSource {
   String READ_REQUEST_COUNT = "readRequestCount";
   String READ_REQUEST_COUNT_DESC =
       "Number of read requests this region server has answered.";
+  String FILTERED_READ_REQUEST_COUNT = "filteredReadRequestCount";
+  String FILTERED_READ_REQUEST_COUNT_DESC =
+    "Number of filtered read requests this region server has answered.";
   String WRITE_REQUEST_COUNT = "writeRequestCount";
   String WRITE_REQUEST_COUNT_DESC =
       "Number of mutation requests this region server has answered.";

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index 02dec8d..07c3773 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@ -113,6 +113,11 @@ public interface MetricsRegionServerWrapper {
   long getReadRequestsCount();
 
   /**
+   * Get the number of filtered read requests to regions hosted on this region server.
+   */
+  long getFilteredReadRequestsCount();
+
+  /**
    * Get the number of write requests to regions hosted on this region server.
    */
   long getWriteRequestsCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
index 0997f7c..20ca9bd 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
@@ -70,6 +70,11 @@ public interface MetricsRegionWrapper {
   long getReadRequestCount();
 
   /**
+   * Get the total number of filtered read requests that have been issued against this region
+   */
+  long getFilteredReadRequestCount();
+
+  /**
    * Get the total number of mutations that have been issued against this region.
    */
   long getWriteRequestCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index f40811c..42476a7 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -201,6 +201,8 @@ public class MetricsRegionServerSourceImpl
               rsWrap.getTotalRequestCount())
           .addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),
               rsWrap.getReadRequestsCount())
+          .addCounter(Interns.info(FILTERED_READ_REQUEST_COUNT, FILTERED_READ_REQUEST_COUNT_DESC),
+              rsWrap.getFilteredReadRequestsCount())
           .addCounter(Interns.info(WRITE_REQUEST_COUNT, WRITE_REQUEST_COUNT_DESC),
               rsWrap.getWriteRequestsCount())
           .addCounter(Interns.info(CHECK_MUTATE_FAILED_COUNT, CHECK_MUTATE_FAILED_COUNT_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index 1df72d5..fab6b51 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -237,6 +237,10 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
               MetricsRegionServerSource.READ_REQUEST_COUNT_DESC),
           this.regionWrapper.getReadRequestCount());
       mrb.addCounter(Interns.info(
+              regionNamePrefix + MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT,
+              MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT_DESC),
+          this.regionWrapper.getFilteredReadRequestCount());
+      mrb.addCounter(Interns.info(
               regionNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
               MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
           this.regionWrapper.getWriteRequestCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
index 3088260..19624aa 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java
@@ -107,6 +107,11 @@ public class TestMetricsRegionSourceImpl {
     }
 
     @Override
+    public long getFilteredReadRequestCount() {
+      return 0;
+    }
+
+    @Override
     public long getWriteRequestCount() {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
index 5fd4e18..048f5cf 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClusterStatusProtos.java
@@ -3613,6 +3613,24 @@ public final class ClusterStatusProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceIdOrBuilder getStoreCompleteSequenceIdOrBuilder(
         int index);
+
+    // optional uint64 filtered_read_requests_count = 19;
+    /**
+     * <code>optional uint64 filtered_read_requests_count = 19;</code>
+     *
+     * <pre>
+     ** the current total filtered read requests made to region 
+     * </pre>
+     */
+    boolean hasFilteredReadRequestsCount();
+    /**
+     * <code>optional uint64 filtered_read_requests_count = 19;</code>
+     *
+     * <pre>
+     ** the current total filtered read requests made to region 
+     * </pre>
+     */
+    long getFilteredReadRequestsCount();
   }
   /**
    * Protobuf type {@code hbase.pb.RegionLoad}
@@ -3766,6 +3784,11 @@ public final class ClusterStatusProtos {
               storeCompleteSequenceId_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId.PARSER, extensionRegistry));
               break;
             }
+            case 152: {
+              bitField0_ |= 0x00020000;
+              filteredReadRequestsCount_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4283,6 +4306,30 @@ public final class ClusterStatusProtos {
       return storeCompleteSequenceId_.get(index);
     }
 
+    // optional uint64 filtered_read_requests_count = 19;
+    public static final int FILTERED_READ_REQUESTS_COUNT_FIELD_NUMBER = 19;
+    private long filteredReadRequestsCount_;
+    /**
+     * <code>optional uint64 filtered_read_requests_count = 19;</code>
+     *
+     * <pre>
+     ** the current total filtered read requests made to region 
+     * </pre>
+     */
+    public boolean hasFilteredReadRequestsCount() {
+      return ((bitField0_ & 0x00020000) == 0x00020000);
+    }
+    /**
+     * <code>optional uint64 filtered_read_requests_count = 19;</code>
+     *
+     * <pre>
+     ** the current total filtered read requests made to region 
+     * </pre>
+     */
+    public long getFilteredReadRequestsCount() {
+      return filteredReadRequestsCount_;
+    }
+
     private void initFields() {
       regionSpecifier_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
       stores_ = 0;
@@ -4302,6 +4349,7 @@ public final class ClusterStatusProtos {
       dataLocality_ = 0F;
       lastMajorCompactionTs_ = 0L;
       storeCompleteSequenceId_ = java.util.Collections.emptyList();
+      filteredReadRequestsCount_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4383,6 +4431,9 @@ public final class ClusterStatusProtos {
       for (int i = 0; i < storeCompleteSequenceId_.size(); i++) {
         output.writeMessage(18, storeCompleteSequenceId_.get(i));
       }
+      if (((bitField0_ & 0x00020000) == 0x00020000)) {
+        output.writeUInt64(19, filteredReadRequestsCount_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4464,6 +4515,10 @@ public final class ClusterStatusProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(18, storeCompleteSequenceId_.get(i));
       }
+      if (((bitField0_ & 0x00020000) == 0x00020000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(19, filteredReadRequestsCount_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4573,6 +4628,11 @@ public final class ClusterStatusProtos {
       }
       result = result && getStoreCompleteSequenceIdList()
           .equals(other.getStoreCompleteSequenceIdList());
+      result = result && (hasFilteredReadRequestsCount() == other.hasFilteredReadRequestsCount());
+      if (hasFilteredReadRequestsCount()) {
+        result = result && (getFilteredReadRequestsCount()
+            == other.getFilteredReadRequestsCount());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4659,6 +4719,10 @@ public final class ClusterStatusProtos {
         hash = (37 * hash) + STORE_COMPLETE_SEQUENCE_ID_FIELD_NUMBER;
         hash = (53 * hash) + getStoreCompleteSequenceIdList().hashCode();
       }
+      if (hasFilteredReadRequestsCount()) {
+        hash = (37 * hash) + FILTERED_READ_REQUESTS_COUNT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getFilteredReadRequestsCount());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4814,6 +4878,8 @@ public final class ClusterStatusProtos {
         } else {
           storeCompleteSequenceIdBuilder_.clear();
         }
+        filteredReadRequestsCount_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00040000);
         return this;
       }
 
@@ -4923,6 +4989,10 @@ public final class ClusterStatusProtos {
         } else {
           result.storeCompleteSequenceId_ = storeCompleteSequenceIdBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00040000) == 0x00040000)) {
+          to_bitField0_ |= 0x00020000;
+        }
+        result.filteredReadRequestsCount_ = filteredReadRequestsCount_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5016,6 +5086,9 @@ public final class ClusterStatusProtos {
             }
           }
         }
+        if (other.hasFilteredReadRequestsCount()) {
+          setFilteredReadRequestsCount(other.getFilteredReadRequestsCount());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -6306,6 +6379,55 @@ public final class ClusterStatusProtos {
         return storeCompleteSequenceIdBuilder_;
       }
 
+      // optional uint64 filtered_read_requests_count = 19;
+      private long filteredReadRequestsCount_ ;
+      /**
+       * <code>optional uint64 filtered_read_requests_count = 19;</code>
+       *
+       * <pre>
+       ** the current total filtered read requests made to region 
+       * </pre>
+       */
+      public boolean hasFilteredReadRequestsCount() {
+        return ((bitField0_ & 0x00040000) == 0x00040000);
+      }
+      /**
+       * <code>optional uint64 filtered_read_requests_count = 19;</code>
+       *
+       * <pre>
+       ** the current total filtered read requests made to region 
+       * </pre>
+       */
+      public long getFilteredReadRequestsCount() {
+        return filteredReadRequestsCount_;
+      }
+      /**
+       * <code>optional uint64 filtered_read_requests_count = 19;</code>
+       *
+       * <pre>
+       ** the current total filtered read requests made to region 
+       * </pre>
+       */
+      public Builder setFilteredReadRequestsCount(long value) {
+        bitField0_ |= 0x00040000;
+        filteredReadRequestsCount_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 filtered_read_requests_count = 19;</code>
+       *
+       * <pre>
+       ** the current total filtered read requests made to region 
+       * </pre>
+       */
+      public Builder clearFilteredReadRequestsCount() {
+        bitField0_ = (bitField0_ & ~0x00040000);
+        filteredReadRequestsCount_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.RegionLoad)
     }
 
@@ -14723,7 +14845,7 @@ public final class ClusterStatusProtos {
       "e\030\001 \002(\014\022\023\n\013sequence_id\030\002 \002(\004\"p\n\026RegionSt" +
       "oreSequenceIds\022 \n\030last_flushed_sequence_" +
       "id\030\001 \002(\004\0224\n\021store_sequence_id\030\002 \003(\0132\031.hb" +
-      "ase.pb.StoreSequenceId\"\324\004\n\nRegionLoad\0223\n" +
+      "ase.pb.StoreSequenceId\"\372\004\n\nRegionLoad\0223\n" +
       "\020region_specifier\030\001 \002(\0132\031.hbase.pb.Regio" +
       "nSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles",
       "\030\003 \001(\r\022\"\n\032store_uncompressed_size_MB\030\004 \001" +
@@ -14738,38 +14860,39 @@ public final class ClusterStatusProtos {
       "_sequence_id\030\017 \001(\004\022\025\n\rdata_locality\030\020 \001(",
       "\002\022#\n\030last_major_compaction_ts\030\021 \001(\004:\0010\022=" +
       "\n\032store_complete_sequence_id\030\022 \003(\0132\031.hba" +
-      "se.pb.StoreSequenceId\"T\n\023ReplicationLoad" +
-      "Sink\022\032\n\022ageOfLastAppliedOp\030\001 \002(\004\022!\n\031time" +
-      "StampsOfLastAppliedOp\030\002 \002(\004\"\225\001\n\025Replicat" +
-      "ionLoadSource\022\016\n\006peerID\030\001 \002(\t\022\032\n\022ageOfLa" +
-      "stShippedOp\030\002 \002(\004\022\026\n\016sizeOfLogQueue\030\003 \002(" +
-      "\r\022 \n\030timeStampOfLastShippedOp\030\004 \002(\004\022\026\n\016r" +
-      "eplicationLag\030\005 \002(\004\"\212\003\n\nServerLoad\022\032\n\022nu" +
-      "mber_of_requests\030\001 \001(\004\022 \n\030total_number_o",
-      "f_requests\030\002 \001(\004\022\024\n\014used_heap_MB\030\003 \001(\r\022\023" +
-      "\n\013max_heap_MB\030\004 \001(\r\022*\n\014region_loads\030\005 \003(" +
-      "\0132\024.hbase.pb.RegionLoad\022+\n\014coprocessors\030" +
-      "\006 \003(\0132\025.hbase.pb.Coprocessor\022\031\n\021report_s" +
-      "tart_time\030\007 \001(\004\022\027\n\017report_end_time\030\010 \001(\004" +
-      "\022\030\n\020info_server_port\030\t \001(\r\0227\n\016replLoadSo" +
-      "urce\030\n \003(\0132\037.hbase.pb.ReplicationLoadSou" +
-      "rce\0223\n\014replLoadSink\030\013 \001(\0132\035.hbase.pb.Rep" +
-      "licationLoadSink\"a\n\016LiveServerInfo\022$\n\006se" +
-      "rver\030\001 \002(\0132\024.hbase.pb.ServerName\022)\n\013serv",
-      "er_load\030\002 \002(\0132\024.hbase.pb.ServerLoad\"\250\003\n\r" +
-      "ClusterStatus\0228\n\rhbase_version\030\001 \001(\0132!.h" +
-      "base.pb.HBaseVersionFileContent\022.\n\014live_" +
-      "servers\030\002 \003(\0132\030.hbase.pb.LiveServerInfo\022" +
-      "*\n\014dead_servers\030\003 \003(\0132\024.hbase.pb.ServerN" +
-      "ame\022;\n\025regions_in_transition\030\004 \003(\0132\034.hba" +
-      "se.pb.RegionInTransition\022\'\n\ncluster_id\030\005" +
-      " \001(\0132\023.hbase.pb.ClusterId\0222\n\023master_copr" +
-      "ocessors\030\006 \003(\0132\025.hbase.pb.Coprocessor\022$\n" +
-      "\006master\030\007 \001(\0132\024.hbase.pb.ServerName\022,\n\016b",
-      "ackup_masters\030\010 \003(\0132\024.hbase.pb.ServerNam" +
-      "e\022\023\n\013balancer_on\030\t \001(\010BF\n*org.apache.had" +
-      "oop.hbase.protobuf.generatedB\023ClusterSta" +
-      "tusProtosH\001\240\001\001"
+      "se.pb.StoreSequenceId\022$\n\034filtered_read_r" +
+      "equests_count\030\023 \001(\004\"T\n\023ReplicationLoadSi" +
+      "nk\022\032\n\022ageOfLastAppliedOp\030\001 \002(\004\022!\n\031timeSt" +
+      "ampsOfLastAppliedOp\030\002 \002(\004\"\225\001\n\025Replicatio" +
+      "nLoadSource\022\016\n\006peerID\030\001 \002(\t\022\032\n\022ageOfLast" +
+      "ShippedOp\030\002 \002(\004\022\026\n\016sizeOfLogQueue\030\003 \002(\r\022" +
+      " \n\030timeStampOfLastShippedOp\030\004 \002(\004\022\026\n\016rep" +
+      "licationLag\030\005 \002(\004\"\212\003\n\nServerLoad\022\032\n\022numb",
+      "er_of_requests\030\001 \001(\004\022 \n\030total_number_of_" +
+      "requests\030\002 \001(\004\022\024\n\014used_heap_MB\030\003 \001(\r\022\023\n\013" +
+      "max_heap_MB\030\004 \001(\r\022*\n\014region_loads\030\005 \003(\0132" +
+      "\024.hbase.pb.RegionLoad\022+\n\014coprocessors\030\006 " +
+      "\003(\0132\025.hbase.pb.Coprocessor\022\031\n\021report_sta" +
+      "rt_time\030\007 \001(\004\022\027\n\017report_end_time\030\010 \001(\004\022\030" +
+      "\n\020info_server_port\030\t \001(\r\0227\n\016replLoadSour" +
+      "ce\030\n \003(\0132\037.hbase.pb.ReplicationLoadSourc" +
+      "e\0223\n\014replLoadSink\030\013 \001(\0132\035.hbase.pb.Repli" +
+      "cationLoadSink\"a\n\016LiveServerInfo\022$\n\006serv",
+      "er\030\001 \002(\0132\024.hbase.pb.ServerName\022)\n\013server" +
+      "_load\030\002 \002(\0132\024.hbase.pb.ServerLoad\"\250\003\n\rCl" +
+      "usterStatus\0228\n\rhbase_version\030\001 \001(\0132!.hba" +
+      "se.pb.HBaseVersionFileContent\022.\n\014live_se" +
+      "rvers\030\002 \003(\0132\030.hbase.pb.LiveServerInfo\022*\n" +
+      "\014dead_servers\030\003 \003(\0132\024.hbase.pb.ServerNam" +
+      "e\022;\n\025regions_in_transition\030\004 \003(\0132\034.hbase" +
+      ".pb.RegionInTransition\022\'\n\ncluster_id\030\005 \001" +
+      "(\0132\023.hbase.pb.ClusterId\0222\n\023master_coproc" +
+      "essors\030\006 \003(\0132\025.hbase.pb.Coprocessor\022$\n\006m",
+      "aster\030\007 \001(\0132\024.hbase.pb.ServerName\022,\n\016bac" +
+      "kup_masters\030\010 \003(\0132\024.hbase.pb.ServerName\022" +
+      "\023\n\013balancer_on\030\t \001(\010BF\n*org.apache.hadoo" +
+      "p.hbase.protobuf.generatedB\023ClusterStatu" +
+      "sProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -14805,7 +14928,7 @@ public final class ClusterStatusProtos {
           internal_static_hbase_pb_RegionLoad_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionLoad_descriptor,
-              new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", });
+              new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", "FilteredReadRequestsCount", });
           internal_static_hbase_pb_ReplicationLoadSink_descriptor =
             getDescriptor().getMessageTypes().get(5);
           internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-protocol/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/ClusterStatus.proto b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
index 228be7e..54bc0c3 100644
--- a/hbase-protocol/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol/src/main/protobuf/ClusterStatus.proto
@@ -136,6 +136,9 @@ message RegionLoad {
 
   /** the most recent sequence Id of store from cache flush */
   repeated StoreSequenceId store_complete_sequence_id = 18;
+
+  /** the current total filtered read requests made to region */
+  optional uint64 filtered_read_requests_count = 19;
 }
 
 /* Server-level protobufs */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
index b7c894f..c051743 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/RegionServerListTmpl.jamon
@@ -174,6 +174,7 @@ for (ServerName serverName: serverNames) {
     <th>ServerName</th>
     <th>Request Per Second</th>
     <th>Read Request Count</th>
+    <th>Filtered Read Request Count</th>
     <th>Write Request Count</th>
 </tr>
 <%java>
@@ -186,6 +187,7 @@ if (sl != null) {
 <td><& serverNameLink; serverName=serverName; serverLoad = sl; &></td>
 <td><% String.format("%.0f", sl.getRequestsPerSecond()) %></td>
 <td><% sl.getReadRequestsCount() %></td>
+<td><% sl.getFilteredReadRequestsCount() %></td>
 <td><% sl.getWriteRequestsCount() %></td>
 </tr>
 <%java>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
index 7740c53..cefd476 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
@@ -118,6 +118,7 @@
         <tr>
             <th>Region Name</th>
             <th>Read Request Count</th>
+            <th>Filtered Read Request Count</th>
             <th>Write Request Count</th>
         </tr>
 
@@ -132,6 +133,7 @@
             <td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
             <%if load != null %>
             <td><% load.getReadRequestsCount() %></td>
+            <td><% load.getFilteredReadRequestsCount() %></td>
             <td><% load.getWriteRequestsCount() %></td>
             </%if>
         </tr>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index 4f6a754..a55a863 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -173,11 +173,13 @@ MetricsRegionServerWrapper mWrap;
 <tr>
     <th>Request Per Second</th>
     <th>Read Request Count</th>
+    <th>Filtered Read Request Count</th>
     <th>Write Request Count</th>
 </tr>
 <tr>
     <td><% String.format("%.0f", mWrap.getRequestsPerSecond()) %></td>
     <td><% mWrap.getReadRequestsCount() %></td>
+    <td><% mWrap.getFilteredReadRequestsCount() %></td>
     <td><% mWrap.getWriteRequestsCount() %></td>
 </tr>
 </table>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/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 99b571f..f03c205 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
@@ -284,6 +284,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   //Number of requests
   final Counter readRequestsCount = new Counter();
+  final Counter filteredReadRequestsCount = new Counter();
   final Counter writeRequestsCount = new Counter();
 
   // Number of requests blocked by memstore size.
@@ -1113,6 +1114,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   @Override
+  public long getFilteredReadRequestsCount() {
+    return filteredReadRequestsCount.get();
+  }
+
+  @Override
   public long getWriteRequestsCount() {
     return writeRequestsCount.get();
   }
@@ -6025,6 +6031,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
 
     protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {
+      filteredReadRequestsCount.increment();
+
       if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
 
       scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();
@@ -6524,6 +6532,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
         this.getBaseConf(), hri, this.getTableDesc(), rsServices);
     r.readRequestsCount.set(this.getReadRequestsCount() / 2);
+    r.filteredReadRequestsCount.set(this.getFilteredReadRequestsCount() / 2);
     r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
     return r;
   }
@@ -6541,6 +6550,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         this.getTableDesc(), this.rsServices);
     r.readRequestsCount.set(this.getReadRequestsCount()
         + region_b.getReadRequestsCount());
+    r.filteredReadRequestsCount.set(this.getFilteredReadRequestsCount()
+      + region_b.getFilteredReadRequestsCount());
     r.writeRequestsCount.set(this.getWriteRequestsCount()
 
         + region_b.getWriteRequestsCount());
@@ -7590,7 +7601,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      44 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      45 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (14 * Bytes.SIZEOF_LONG) +
       5 * Bytes.SIZEOF_BOOLEAN);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 1183f96..9cb100f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1518,6 +1518,7 @@ public class HRegionServer extends HasThread implements
       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
       .setReadRequestsCount(r.getReadRequestsCount())
+      .setFilteredReadRequestsCount(r.getFilteredReadRequestsCount())
       .setWriteRequestsCount(r.getWriteRequestsCount())
       .setTotalCompactingKVs(totalCompactingKVs)
       .setCurrentCompactedKVs(currentCompactedKVs)

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index f3e8916..4f9ba5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -68,6 +68,7 @@ class MetricsRegionServerWrapperImpl
   private volatile long storeFileSize = 0;
   private volatile double requestsPerSecond = 0.0;
   private volatile long readRequestsCount = 0;
+  private volatile long filteredReadRequestsCount = 0;
   private volatile long writeRequestsCount = 0;
   private volatile long checkAndMutateChecksFailed = 0;
   private volatile long checkAndMutateChecksPassed = 0;
@@ -408,6 +409,11 @@ class MetricsRegionServerWrapperImpl
   }
 
   @Override
+  public long getFilteredReadRequestsCount() {
+    return filteredReadRequestsCount;
+  }
+
+  @Override
   public long getWriteRequestsCount() {
     return writeRequestsCount;
   }
@@ -588,7 +594,8 @@ class MetricsRegionServerWrapperImpl
             new HDFSBlocksDistribution();
 
         long tempNumStores = 0, tempNumStoreFiles = 0, tempMemstoreSize = 0, tempStoreFileSize = 0;
-        long tempReadRequestsCount = 0, tempWriteRequestsCount = 0;
+        long tempReadRequestsCount = 0, tempFilteredReadRequestsCount = 0,
+          tempWriteRequestsCount = 0;
         long tempCheckAndMutateChecksFailed = 0;
         long tempCheckAndMutateChecksPassed = 0;
         long tempStorefileIndexSize = 0;
@@ -619,6 +626,7 @@ class MetricsRegionServerWrapperImpl
           tempNumMutationsWithoutWAL += r.getNumMutationsWithoutWAL();
           tempDataInMemoryWithoutWAL += r.getDataInMemoryWithoutWAL();
           tempReadRequestsCount += r.getReadRequestsCount();
+          tempFilteredReadRequestsCount += r.getFilteredReadRequestsCount();
           tempWriteRequestsCount += r.getWriteRequestsCount();
           tempCheckAndMutateChecksFailed += r.getCheckAndMutateChecksFailed();
           tempCheckAndMutateChecksPassed += r.getCheckAndMutateChecksPassed();
@@ -696,6 +704,7 @@ class MetricsRegionServerWrapperImpl
         memstoreSize = tempMemstoreSize;
         storeFileSize = tempStoreFileSize;
         readRequestsCount = tempReadRequestsCount;
+        filteredReadRequestsCount = tempFilteredReadRequestsCount;
         writeRequestsCount = tempWriteRequestsCount;
         checkAndMutateChecksFailed = tempCheckAndMutateChecksFailed;
         checkAndMutateChecksPassed = tempCheckAndMutateChecksPassed;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
index 08865e6..2c54079 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java
@@ -112,6 +112,11 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
   }
 
   @Override
+  public long getFilteredReadRequestCount() {
+    return this.region.getFilteredReadRequestsCount();
+  }
+
+  @Override
   public long getWriteRequestCount() {
     return this.region.getWriteRequestsCount();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 213b41a..976bddb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -185,6 +185,9 @@ public interface Region extends ConfigurationObserver {
    */
   void updateReadRequestsCount(long i);
 
+  /** @return filtered read requests count for this region */
+  long getFilteredReadRequestsCount();
+
   /** @return write request count for this region */
   long getWriteRequestsCount();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionServerMetrics.java
new file mode 100644
index 0000000..76e5842
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionServerMetrics.java
@@ -0,0 +1,379 @@
+/**
+ *
+ * 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 org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+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.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(MediumTests.class)
+public class TestRegionServerMetrics {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final TableName TABLE_NAME = TableName.valueOf("test");
+  private static final byte[] CF1 = "c1".getBytes();
+  private static final byte[] CF2 = "c2".getBytes();
+
+  private static final byte[] ROW1 = "a".getBytes();
+  private static final byte[] ROW2 = "b".getBytes();
+  private static final byte[] ROW3 = "c".getBytes();
+  private static final byte[] COL1 = "q1".getBytes();
+  private static final byte[] COL2 = "q2".getBytes();
+  private static final byte[] COL3 = "q3".getBytes();
+  private static final byte[] VAL1 = "v1".getBytes();
+  private static final byte[] VAL2 = "v2".getBytes();
+  private static final byte[] VAL3 = Bytes.toBytes(0L);
+
+  private static final int MAX_TRY = 20;
+  private static final int SLEEP_MS = 100;
+  private static final int TTL = 1;
+
+  private static Admin admin;
+  private static Collection<ServerName> serverNames;
+  private static Table table;
+  private static List<HRegionInfo> tableRegions;
+
+  private static Map<Metric, Long> requestsMap = new HashMap<>();
+  private static Map<Metric, Long> requestsMapPrev = new HashMap<>();
+
+  @BeforeClass
+  public static void setUpOnce() throws Exception {
+    TEST_UTIL.startMiniCluster();
+    admin = TEST_UTIL.getAdmin();
+    serverNames = admin.getClusterStatus().getServers();
+    table = createTable();
+    putData();
+    tableRegions = admin.getTableRegions(TABLE_NAME);
+
+    for (Metric metric : Metric.values()) {
+      requestsMap.put(metric, 0L);
+      requestsMapPrev.put(metric, 0L);
+    }
+  }
+
+  private static Table createTable() throws IOException {
+    HTableDescriptor td = new HTableDescriptor(TABLE_NAME);
+    HColumnDescriptor cd1 = new HColumnDescriptor(CF1);
+    td.addFamily(cd1);
+    HColumnDescriptor cd2 = new HColumnDescriptor(CF2);
+    cd2.setTimeToLive(TTL);
+    td.addFamily(cd2);
+
+    admin.createTable(td);
+    return TEST_UTIL.getConnection().getTable(TABLE_NAME);
+  }
+
+  private static void testReadRequests(long resultCount,
+    long expectedReadRequests, long expectedFilteredReadRequests)
+    throws IOException, InterruptedException {
+    updateMetricsMap();
+    System.out.println("requestsMapPrev = " + requestsMapPrev);
+    System.out.println("requestsMap = " + requestsMap);
+
+    assertEquals(expectedReadRequests,
+      requestsMap.get(Metric.REGION_READ) - requestsMapPrev.get(Metric.REGION_READ));
+    assertEquals(expectedReadRequests,
+      requestsMap.get(Metric.SERVER_READ) - requestsMapPrev.get(Metric.SERVER_READ));
+    assertEquals(expectedFilteredReadRequests,
+      requestsMap.get(Metric.FILTERED_REGION_READ)
+        - requestsMapPrev.get(Metric.FILTERED_REGION_READ));
+    assertEquals(expectedFilteredReadRequests,
+      requestsMap.get(Metric.FILTERED_SERVER_READ)
+        - requestsMapPrev.get(Metric.FILTERED_SERVER_READ));
+    assertEquals(expectedReadRequests, resultCount);
+  }
+
+  private static void updateMetricsMap() throws IOException, InterruptedException {
+    for (Metric metric : Metric.values()) {
+      requestsMapPrev.put(metric, requestsMap.get(metric));
+    }
+
+    ServerLoad serverLoad = null;
+    RegionLoad regionLoadOuter = null;
+    boolean metricsUpdated = false;
+    for (int i = 0; i < MAX_TRY; i++) {
+      for (ServerName serverName : serverNames) {
+        serverLoad = admin.getClusterStatus().getLoad(serverName);
+
+        Map<byte[], RegionLoad> regionsLoad = serverLoad.getRegionsLoad();
+        for (HRegionInfo tableRegion : tableRegions) {
+          RegionLoad regionLoad = regionsLoad.get(tableRegion.getRegionName());
+          if (regionLoad != null) {
+            regionLoadOuter = regionLoad;
+            for (Metric metric : Metric.values()) {
+              if (getReadRequest(serverLoad, regionLoad, metric) > requestsMapPrev.get(metric)) {
+                for (Metric metricInner : Metric.values()) {
+                  requestsMap.put(metricInner, getReadRequest(serverLoad, regionLoad, metricInner));
+                }
+                metricsUpdated = true;
+                break;
+              }
+            }
+          }
+        }
+      }
+      if (metricsUpdated) {
+        break;
+      }
+      Thread.sleep(SLEEP_MS);
+    }
+    if (!metricsUpdated) {
+      for (Metric metric : Metric.values()) {
+        requestsMap.put(metric, getReadRequest(serverLoad, regionLoadOuter, metric));
+      }
+    }
+  }
+
+  private static long getReadRequest(ServerLoad serverLoad, RegionLoad regionLoad, Metric metric) {
+    switch (metric) {
+      case REGION_READ:
+        return regionLoad.getReadRequestsCount();
+      case SERVER_READ:
+        return serverLoad.getReadRequestsCount();
+      case FILTERED_REGION_READ:
+        return regionLoad.getFilteredReadRequestsCount();
+      case FILTERED_SERVER_READ:
+        return serverLoad.getFilteredReadRequestsCount();
+      default:
+        throw new IllegalStateException();
+    }
+  }
+
+  private static void putData() throws IOException {
+    Put put;
+
+    put = new Put(ROW1);
+    put.addColumn(CF1, COL1, VAL1);
+    put.addColumn(CF1, COL2, VAL2);
+    put.addColumn(CF1, COL3, VAL3);
+    table.put(put);
+    put = new Put(ROW2);
+    put.addColumn(CF1, COL1, VAL2);  // put val2 instead of val1
+    put.addColumn(CF1, COL2, VAL2);
+    table.put(put);
+    put = new Put(ROW3);
+    put.addColumn(CF1, COL1, VAL1);
+    put.addColumn(CF1, COL2, VAL2);
+    table.put(put);
+  }
+
+  private static void putTTLExpiredData() throws IOException, InterruptedException {
+    Put put;
+
+    put = new Put(ROW1);
+    put.addColumn(CF2, COL1, VAL1);
+    put.addColumn(CF2, COL2, VAL2);
+    table.put(put);
+
+    Thread.sleep(TTL * 1000);
+
+    put = new Put(ROW2);
+    put.addColumn(CF2, COL1, VAL1);
+    put.addColumn(CF2, COL2, VAL2);
+    table.put(put);
+
+    put = new Put(ROW3);
+    put.addColumn(CF2, COL1, VAL1);
+    put.addColumn(CF2, COL2, VAL2);
+    table.put(put);
+  }
+
+  @AfterClass
+  public static void tearDownOnce() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testReadRequestsCountNotFiltered() throws Exception {
+    int resultCount;
+    Scan scan;
+    Append append;
+    Put put;
+    Increment increment;
+    Get get;
+
+    // test for scan
+    scan = new Scan();
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 3, 0);
+    }
+
+    // test for scan
+    scan = new Scan(ROW2, ROW3);
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 1, 0);
+    }
+
+    // test for get
+    get = new Get(ROW2);
+    Result result = table.get(get);
+    resultCount = result.isEmpty() ? 0 : 1;
+    testReadRequests(resultCount, 1, 0);
+
+    // test for increment
+    increment = new Increment(ROW1);
+    increment.addColumn(CF1, COL3, 1);
+    result = table.increment(increment);
+    resultCount = result.isEmpty() ? 0 : 1;
+    testReadRequests(resultCount, 1, 0);
+
+    // test for checkAndPut
+    put = new Put(ROW1);
+    put.addColumn(CF1, COL2, VAL2);
+    boolean checkAndPut =
+      table.checkAndPut(ROW1, CF1, COL2, CompareFilter.CompareOp.EQUAL, VAL2, put);
+    resultCount = checkAndPut ? 1 : 0;
+    testReadRequests(resultCount, 1, 0);
+
+    // test for append
+    append = new Append(ROW1);
+    append.add(CF1, COL2, VAL2);
+    result = table.append(append);
+    resultCount = result.isEmpty() ? 0 : 1;
+    testReadRequests(resultCount, 1, 0);
+
+    // test for checkAndMutate
+    put = new Put(ROW1);
+    put.addColumn(CF1, COL1, VAL1);
+    RowMutations rm = new RowMutations(ROW1);
+    rm.add(put);
+    boolean checkAndMutate =
+      table.checkAndMutate(ROW1, CF1, COL1, CompareFilter.CompareOp.EQUAL, VAL1, rm);
+    resultCount = checkAndMutate ? 1 : 0;
+    testReadRequests(resultCount, 1, 0);
+  }
+
+  @Test
+  public void testReadRequestsCountWithFilter() throws Exception {
+    int resultCount;
+    Scan scan;
+
+    // test for scan
+    scan = new Scan();
+    scan.setFilter(new SingleColumnValueFilter(CF1, COL1, CompareFilter.CompareOp.EQUAL, VAL1));
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 2, 1);
+    }
+
+    // test for scan
+    scan = new Scan();
+    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(ROW1)));
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 1, 2);
+    }
+
+    // test for scan
+    scan = new Scan(ROW2, ROW3);
+    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, new BinaryComparator(ROW1)));
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 0, 1);
+    }
+
+    // fixme filtered get should not increase readRequestsCount
+//    Get get = new Get(ROW2);
+//    get.setFilter(new SingleColumnValueFilter(CF1, COL1, CompareFilter.CompareOp.EQUAL, VAL1));
+//    Result result = table.get(get);
+//    resultCount = result.isEmpty() ? 0 : 1;
+//    testReadRequests(resultCount, 0, 1);
+  }
+
+  @Test
+  public void testReadRequestsCountWithDeletedRow() throws Exception {
+    try {
+      Delete delete = new Delete(ROW3);
+      table.delete(delete);
+
+      Scan scan = new Scan();
+      try (ResultScanner scanner = table.getScanner(scan)) {
+        int resultCount = 0;
+        for (Result ignore : scanner) {
+          resultCount++;
+        }
+        testReadRequests(resultCount, 2, 1);
+      }
+    } finally {
+      Put put = new Put(ROW3);
+      put.addColumn(CF1, COL1, VAL1);
+      put.addColumn(CF1, COL2, VAL2);
+      table.put(put);
+    }
+  }
+
+  @Test
+  public void testReadRequestsCountWithTTLExpiration() throws Exception {
+    putTTLExpiredData();
+
+    Scan scan = new Scan();
+    scan.addFamily(CF2);
+    try (ResultScanner scanner = table.getScanner(scan)) {
+      int resultCount = 0;
+      for (Result ignore : scanner) {
+        resultCount++;
+      }
+      testReadRequests(resultCount, 2, 1);
+    }
+  }
+
+  private enum Metric {REGION_READ, SERVER_READ, FILTERED_REGION_READ, FILTERED_SERVER_READ}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
index 5c56e9a..e6c17a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
@@ -44,6 +44,7 @@ public class TestServerLoad {
     assertEquals(820, sl.getStorefileSizeInMB());
     assertEquals(82, sl.getStorefileIndexSizeInMB());
     assertEquals(((long)Integer.MAX_VALUE)*2, sl.getReadRequestsCount());
+    assertEquals(300, sl.getFilteredReadRequestsCount());
     
   }
  
@@ -57,6 +58,7 @@ public class TestServerLoad {
     assertTrue(slToString.contains("storefileSizeMB=820")); 
     assertTrue(slToString.contains("rootIndexSizeKB=504"));
     assertTrue(slToString.contains("coprocessors=[]"));
+    assertTrue(slToString.contains("filteredReadRequestsCount=300"));
   }
 
   @Test
@@ -80,10 +82,12 @@ public class TestServerLoad {
     ClusterStatusProtos.RegionLoad rlOne =
         ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
             .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
+            .setFilteredReadRequestsCount(100)
             .setStorefileIndexSizeMB(42).setRootIndexSizeKB(201).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
     ClusterStatusProtos.RegionLoad rlTwo =
         ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
             .setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
+            .setFilteredReadRequestsCount(200)
             .setStorefileIndexSizeMB(40).setRootIndexSizeKB(303).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
 
     ClusterStatusProtos.ServerLoad sl =

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index 0d93284..4b00632 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@ -86,6 +86,11 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   }
 
   @Override
+  public long getFilteredReadRequestsCount() {
+    return 1997;
+  }
+
+  @Override
   public long getWriteRequestsCount() {
     return 707;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
index c43ccc3..8e6dd74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java
@@ -76,6 +76,11 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
   }
 
   @Override
+  public long getFilteredReadRequestCount() {
+    return 107;
+  }
+
+  @Override
   public long getWriteRequestCount() {
     return 106;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
index e739890..cc09d15 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
@@ -46,6 +46,10 @@ public class TestMetricsRegion {
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_memstoreSize",
       103, agg);
     HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_" +
+        "filteredReadRequestCount",
+      107, agg);
+    HELPER.assertCounter(
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", 
       0, agg);
     mr.close();
@@ -63,6 +67,10 @@ public class TestMetricsRegion {
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_memstoreSize",
       103, agg);
     HELPER.assertCounter(
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_" +
+        "filteredReadRequestCount",
+      107, agg);
+    HELPER.assertCounter(
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", 
       1, agg);
     mr.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f20bc74/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
index f3ce0bd..77d6a95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java
@@ -67,6 +67,7 @@ public class TestMetricsRegionServer {
     HELPER.assertGauge("storeFileSize", 1900, serverSource);
     HELPER.assertCounter("totalRequestCount", 899, serverSource);
     HELPER.assertCounter("readRequestCount", 997, serverSource);
+    HELPER.assertCounter("filteredReadRequestCount", 1997, serverSource);
     HELPER.assertCounter("writeRequestCount", 707, serverSource);
     HELPER.assertCounter("checkMutateFailedCount", 401, serverSource);
     HELPER.assertCounter("checkMutatePassedCount", 405, serverSource);


[06/32] hbase git commit: HBASE-15177 Reduce garbage created under high load

Posted by sy...@apache.org.
HBASE-15177 Reduce garbage created under high load


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

Branch: refs/heads/hbase-12439
Commit: a69272efe12f7b780fbf2fa14c42d0c0b155205f
Parents: d5d26f0
Author: Enis Soztutar <en...@apache.org>
Authored: Thu Feb 4 11:07:36 2016 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Thu Feb 4 13:27:00 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/ScannerCallable.java    |  14 ++-
 .../hadoop/hbase/ipc/AsyncRpcChannel.java       |   7 +-
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    |  20 ++--
 .../hbase/ipc/PayloadCarryingRpcController.java |   7 +-
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  |   6 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  19 +++-
 .../hadoop/hbase/client/TestClientScanner.java  |   2 +-
 .../apache/hadoop/hbase/ipc/TestIPCUtil.java    |   4 +-
 .../hadoop/hbase/io/ByteBufferInputStream.java  | 107 +++++++++++++++++++
 .../org/apache/hadoop/hbase/util/Threads.java   |   2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  54 ++++++----
 .../AnnotationReadingPriorityFunction.java      |   9 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   6 +-
 .../hbase/regionserver/RSRpcServices.java       |  15 ++-
 14 files changed, 210 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index f6445a6..72d69ec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -191,6 +191,13 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     if (Thread.interrupted()) {
       throw new InterruptedIOException();
     }
+
+    if (controller == null) {
+      controller = controllerFactory.newController();
+      controller.setPriority(getTableName());
+      controller.setCallTimeout(callTimeout);
+    }
+
     if (closed) {
       if (scannerId != -1) {
         close();
@@ -209,9 +216,6 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
               RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
                 this.scanMetrics != null, renew);
           ScanResponse response = null;
-          controller = controllerFactory.newController();
-          controller.setPriority(getTableName());
-          controller.setCallTimeout(callTimeout);
           try {
             response = getStub().scan(controller, request);
             // Client and RS maintain a nextCallSeq number during the scan. Every next() call
@@ -371,7 +375,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
       ScanRequest request =
           RequestConverter.buildScanRequest(this.scannerId, 0, true, this.scanMetrics != null);
       try {
-        getStub().scan(null, request);
+        getStub().scan(controller, request);
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
       }
@@ -388,7 +392,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
         getLocation().getRegionInfo().getRegionName(),
         this.scan, 0, false);
     try {
-      ScanResponse response = getStub().scan(null, request);
+      ScanResponse response = getStub().scan(controller, request);
       long id = response.getScannerId();
       if (logScannerActivity) {
         LOG.info("Open scanner=" + id + " for scan=" + scan.toString()

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
index 69978fc..787aa47 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
@@ -412,7 +412,7 @@ public class AsyncRpcChannel {
         requestHeaderBuilder.setCellBlockMeta(cellBlockBuilder.build());
       }
       // Only pass priority if there one.  Let zero be same as no priority.
-      if (call.controller.getPriority() != 0) {
+      if (call.controller.getPriority() != PayloadCarryingRpcController.PRIORITY_UNSET) {
         requestHeaderBuilder.setPriority(call.controller.getPriority());
       }
 
@@ -660,6 +660,7 @@ public class AsyncRpcChannel {
   private void handleSaslConnectionFailure(final int currRetries, final Throwable ex,
       final UserGroupInformation user) throws IOException, InterruptedException {
     user.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
       public Void run() throws IOException, InterruptedException {
         if (shouldAuthenticateOverKrb()) {
           if (currRetries < MAX_SASL_RETRIES) {
@@ -702,12 +703,12 @@ public class AsyncRpcChannel {
   public int getConnectionHashCode() {
     return ConnectionId.hashCode(ticket, serviceName, address);
   }
-  
+
   @Override
   public int hashCode() {
     return getConnectionHashCode();
   }
-     
+
   @Override
   public boolean equals(Object obj) {
     if (obj instanceof AsyncRpcChannel) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 734227c..22c5cc1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.IOException;
 import java.io.InputStream;
@@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -180,19 +180,18 @@ public class IPCUtil {
   public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
       final byte [] cellBlock)
   throws IOException {
-    return createCellScanner(codec, compressor, cellBlock, 0, cellBlock.length);
+    return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock));
   }
 
   /**
    * @param codec
-   * @param cellBlock
-   * @param offset
-   * @param length
+   * @param cellBlock ByteBuffer containing the cells written by the Codec. The buffer should be
+   * position()'ed at the start of the cell block and limit()'ed at the end.
    * @return CellScanner to work against the content of <code>cellBlock</code>
    * @throws IOException
    */
   public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
-      final byte [] cellBlock, final int offset, final int length)
+      final ByteBuffer cellBlock)
   throws IOException {
     // If compressed, decompress it first before passing it on else we will leak compression
     // resources if the stream is not closed properly after we let it out.
@@ -202,18 +201,17 @@ public class IPCUtil {
       if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
       Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
       CompressionInputStream cis =
-        compressor.createInputStream(new ByteArrayInputStream(cellBlock, offset, length),
-        poolDecompressor);
+        compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor);
       ByteBufferOutputStream bbos = null;
       try {
         // TODO: This is ugly.  The buffer will be resized on us if we guess wrong.
         // TODO: Reuse buffers.
-        bbos = new ByteBufferOutputStream((length - offset) *
+        bbos = new ByteBufferOutputStream(cellBlock.remaining() *
           this.cellBlockDecompressionMultiplier);
         IOUtils.copy(cis, bbos);
         bbos.close();
         ByteBuffer bb = bbos.getByteBuffer();
-        is = new ByteArrayInputStream(bb.array(), 0, bb.limit());
+        is = new ByteBufferInputStream(bb);
       } finally {
         if (is != null) is.close();
         if (bbos != null) bbos.close();
@@ -221,7 +219,7 @@ public class IPCUtil {
         CodecPool.returnDecompressor(poolDecompressor);
       }
     } else {
-      is = new ByteArrayInputStream(cellBlock, offset, length);
+      is = new ByteBufferInputStream(cellBlock);
     }
     return codec.getDecoder(is);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
index 09f4323..f4f18b3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/PayloadCarryingRpcController.java
@@ -35,14 +35,14 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 @InterfaceAudience.Private
 public class PayloadCarryingRpcController
     extends TimeLimitedRpcController implements CellScannable {
+
+  public static final int PRIORITY_UNSET = -1;
   /**
    * Priority to set on this request.  Set it here in controller so available composing the
    * request.  This is the ordained way of setting priorities going forward.  We will be
    * undoing the old annotation-based mechanism.
    */
-  // Currently only multi call makes use of this.  Eventually this should be only way to set
-  // priority.
-  private int priority = HConstants.NORMAL_QOS;
+  private int priority = PRIORITY_UNSET;
 
   /**
    * They are optionally set on construction, cleared after we make the call, and then optionally
@@ -67,6 +67,7 @@ public class PayloadCarryingRpcController
   /**
    * @return One-shot cell scanner (you cannot back it up and restart)
    */
+  @Override
   public CellScanner cellScanner() {
     return cellScanner;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
index 940fcd1..83d4adf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
@@ -899,8 +899,10 @@ public class RpcClientImpl extends AbstractRpcClient {
         cellBlockBuilder.setLength(cellBlock.limit());
         builder.setCellBlockMeta(cellBlockBuilder.build());
       }
-      // Only pass priority if there one.  Let zero be same as no priority.
-      if (priority != 0) builder.setPriority(priority);
+      // Only pass priority if there is one set.
+      if (priority != PayloadCarryingRpcController.PRIORITY_UNSET) {
+        builder.setPriority(priority);
+      }
       RequestHeader header = builder.build();
 
       setupIOstreams();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 7cd0d91..fe76780 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -2430,13 +2430,13 @@ public final class ProtobufUtil {
    */
   public static String getRegionEncodedName(
       final RegionSpecifier regionSpecifier) throws DoNotRetryIOException {
-    byte[] value = regionSpecifier.getValue().toByteArray();
+    ByteString value = regionSpecifier.getValue();
     RegionSpecifierType type = regionSpecifier.getType();
     switch (type) {
       case REGION_NAME:
-        return HRegionInfo.encodeRegionName(value);
+        return HRegionInfo.encodeRegionName(value.toByteArray());
       case ENCODED_REGION_NAME:
-        return Bytes.toString(value);
+        return value.toStringUtf8();
       default:
         throw new DoNotRetryIOException(
           "Unsupported region specifier type: " + type);
@@ -3135,6 +3135,19 @@ public final class ProtobufUtil {
     codedInput.checkLastTagWas(0);
   }
 
+  public static void mergeFrom(Message.Builder builder, CodedInputStream codedInput, int length)
+      throws IOException {
+    codedInput.resetSizeCounter();
+    int prevLimit = codedInput.setSizeLimit(length);
+
+    int limit = codedInput.pushLimit(length);
+    builder.mergeFrom(codedInput);
+    codedInput.popLimit(limit);
+
+    codedInput.checkLastTagWas(0);
+    codedInput.setSizeLimit(prevLimit);
+  }
+
   public static ReplicationLoadSink toReplicationLoadSink(
       ClusterStatusProtos.ReplicationLoadSink cls) {
     return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
index 6d7cc7f..f083001 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java
@@ -522,7 +522,7 @@ public class TestClientScanner {
       anyBoolean(), anyInt())).thenReturn(new RegionLocations(null, null, null));
 
     try (MockClientScanner scanner = new MockClientScanner(conf, scan, TableName.valueOf("table"),
-      clusterConn, rpcFactory, controllerFactory, pool, Integer.MAX_VALUE)) {
+      clusterConn, rpcFactory, new RpcControllerFactory(conf), pool, Integer.MAX_VALUE)) {
       Iterator<Result> iter = scanner.iterator();
       while (iter.hasNext()) {
         iter.next();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
index 163be70..0038aec 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
@@ -58,7 +58,7 @@ public class TestIPCUtil {
   public void before() {
     this.util = new IPCUtil(new Configuration());
   }
-  
+
   @Test
   public void testBuildCellBlock() throws IOException {
     doBuildCellBlockUndoCellBlock(this.util, new KeyValueCodec(), null);
@@ -79,7 +79,7 @@ public class TestIPCUtil {
     CellScanner cellScanner = sized? getSizedCellScanner(cells):
       CellUtil.createCellScanner(Arrays.asList(cells).iterator());
     ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner);
-    cellScanner = util.createCellScanner(codec, compressor, bb.array(), 0, bb.limit());
+    cellScanner = util.createCellScanner(codec, compressor, bb);
     int i = 0;
     while (cellScanner.advance()) {
       i++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
new file mode 100644
index 0000000..8aee07b
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/ByteBufferInputStream.java
@@ -0,0 +1,107 @@
+/**
+ * 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;
+
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * Not thread safe!
+ * <p>
+ * Please note that the reads will cause position movement on wrapped ByteBuffer.
+ */
+@InterfaceAudience.Private
+public class ByteBufferInputStream extends InputStream {
+
+  private ByteBuffer buf;
+
+  public ByteBufferInputStream(ByteBuffer buf) {
+    this.buf = buf;
+  }
+
+  /**
+   * Reads the next byte of data from this input stream. The value byte is returned as an
+   * <code>int</code> in the range <code>0</code> to <code>255</code>. If no byte is available
+   * because the end of the stream has been reached, the value <code>-1</code> is returned.
+   * @return the next byte of data, or <code>-1</code> if the end of the stream has been reached.
+   */
+  @Override
+  public int read() {
+    if (this.buf.hasRemaining()) {
+      return (this.buf.get() & 0xff);
+    }
+    return -1;
+  }
+
+  /**
+   * Reads up to next <code>len</code> bytes of data from buffer into passed array(starting from
+   * given offset).
+   * @param b the array into which the data is read.
+   * @param off the start offset in the destination array <code>b</code>
+   * @param len the maximum number of bytes to read.
+   * @return the total number of bytes actually read into the buffer, or <code>-1</code> if not even
+   *         1 byte can be read because the end of the stream has been reached.
+   */
+  @Override
+  public int read(byte[] b, int off, int len) {
+    int avail = available();
+    if (avail <= 0) {
+      return -1;
+    }
+
+    if (len > avail) {
+      len = avail;
+    }
+    if (len <= 0) {
+      return 0;
+    }
+
+    ByteBufferUtils.copyFromBufferToArray(b, this.buf, this.buf.position(), off, len);
+    this.buf.position(this.buf.position() + len); // we should advance the buffer position
+    return len;
+  }
+
+  /**
+   * Skips <code>n</code> bytes of input from this input stream. Fewer bytes might be skipped if the
+   * end of the input stream is reached. The actual number <code>k</code> of bytes to be skipped is
+   * equal to the smaller of <code>n</code> and remaining bytes in the stream.
+   * @param n the number of bytes to be skipped.
+   * @return the actual number of bytes skipped.
+   */
+  @Override
+  public long skip(long n) {
+    long k = Math.min(n, available());
+    if (k < 0) {
+      k = 0;
+    }
+    this.buf.position((int) (this.buf.position() + k));
+    return k;
+  }
+
+  /**
+   * @return  the number of remaining bytes that can be read (or skipped
+   *          over) from this input stream.
+   */
+  @Override
+  public int available() {
+    return this.buf.remaining();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
index c366762..d10e0f2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Threads.java
@@ -45,7 +45,7 @@ public class Threads {
   private static final Log LOG = LogFactory.getLog(Threads.class);
   private static final AtomicInteger poolNumber = new AtomicInteger(1);
 
-  private static UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER =
+  public static final UncaughtExceptionHandler LOGGING_EXCEPTION_HANDLER =
     new UncaughtExceptionHandler() {
     @Override
     public void uncaughtException(Thread t, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/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 98669e9..58fc598 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
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 import org.apache.hadoop.hbase.io.BoundedByteBufferPool;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -110,6 +111,7 @@ import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
@@ -529,10 +531,12 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       return this.size;
     }
 
+    @Override
     public long getResponseCellSize() {
       return responseCellSize;
     }
 
+    @Override
     public void incrementResponseCellSize(long cellSize) {
       responseCellSize += cellSize;
     }
@@ -621,7 +625,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       readPool = Executors.newFixedThreadPool(readThreads,
         new ThreadFactoryBuilder().setNameFormat(
           "RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
-          ",port=" + port).setDaemon(true).build());
+          ",port=" + port).setDaemon(true)
+        .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
       for (int i = 0; i < readThreads; ++i) {
         Reader reader = new Reader();
         readers[i] = reader;
@@ -898,7 +903,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         throw ieo;
       } catch (Exception e) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug(getName() + ": Caught exception while reading:" + e.getMessage());
+          LOG.debug(getName() + ": Caught exception while reading:", e);
         }
         count = -1; //so that the (count < 0) block is executed
       }
@@ -944,6 +949,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     Responder() throws IOException {
       this.setName("RpcServer.responder");
       this.setDaemon(true);
+      this.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER);
       writeSelector = Selector.open(); // create a selector
     }
 
@@ -1361,17 +1367,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       return authorizedUgi;
     }
 
-    private void saslReadAndProcess(byte[] saslToken) throws IOException,
+    private void saslReadAndProcess(ByteBuffer saslToken) throws IOException,
         InterruptedException {
       if (saslContextEstablished) {
         if (LOG.isTraceEnabled())
-          LOG.trace("Have read input token of size " + saslToken.length
+          LOG.trace("Have read input token of size " + saslToken.limit()
               + " for processing by saslServer.unwrap()");
 
         if (!useWrap) {
           processOneRpc(saslToken);
         } else {
-          byte [] plaintextData = saslServer.unwrap(saslToken, 0, saslToken.length);
+          byte[] b = saslToken.array();
+          byte [] plaintextData = saslServer.unwrap(b, saslToken.position(), saslToken.limit());
           processUnwrappedData(plaintextData);
         }
       } else {
@@ -1420,10 +1427,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
             }
           }
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Have read input token of size " + saslToken.length
+            LOG.debug("Have read input token of size " + saslToken.limit()
                 + " for processing by saslServer.evaluateResponse()");
           }
-          replyToken = saslServer.evaluateResponse(saslToken);
+          replyToken = saslServer.evaluateResponse(saslToken.array());
         } catch (IOException e) {
           IOException sendToClient = e;
           Throwable cause = e;
@@ -1619,6 +1626,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           throw new IllegalArgumentException("Unexpected data length "
               + dataLength + "!! from " + getHostAddress());
         }
+
+       // TODO: check dataLength against some limit so that the client cannot OOM the server
         data = ByteBuffer.allocate(dataLength);
 
         // Increment the rpc count. This counter will be decreased when we write
@@ -1648,9 +1657,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
 
         if (useSasl) {
-          saslReadAndProcess(data.array());
+          saslReadAndProcess(data);
         } else {
-          processOneRpc(data.array());
+          processOneRpc(data);
         }
 
       } finally {
@@ -1679,8 +1688,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     }
 
     // Reads the connection header following version
-    private void processConnectionHeader(byte[] buf) throws IOException {
-      this.connectionHeader = ConnectionHeader.parseFrom(buf);
+    private void processConnectionHeader(ByteBuffer buf) throws IOException {
+      this.connectionHeader = ConnectionHeader.parseFrom(
+        new ByteBufferInputStream(buf));
       String serviceName = connectionHeader.getServiceName();
       if (serviceName == null) throw new EmptyServiceNameException();
       this.service = getService(services, serviceName);
@@ -1794,13 +1804,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         if (unwrappedData.remaining() == 0) {
           unwrappedDataLengthBuffer.clear();
           unwrappedData.flip();
-          processOneRpc(unwrappedData.array());
+          processOneRpc(unwrappedData);
           unwrappedData = null;
         }
       }
     }
 
-    private void processOneRpc(byte[] buf) throws IOException, InterruptedException {
+
+    private void processOneRpc(ByteBuffer buf) throws IOException, InterruptedException {
       if (connectionHeaderRead) {
         processRequest(buf);
       } else {
@@ -1822,16 +1833,16 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
      * @throws IOException
      * @throws InterruptedException
      */
-    protected void processRequest(byte[] buf) throws IOException, InterruptedException {
-      long totalRequestSize = buf.length;
+    protected void processRequest(ByteBuffer buf) throws IOException, InterruptedException {
+      long totalRequestSize = buf.limit();
       int offset = 0;
       // Here we read in the header.  We avoid having pb
       // do its default 4k allocation for CodedInputStream.  We force it to use backing array.
-      CodedInputStream cis = CodedInputStream.newInstance(buf, offset, buf.length);
+      CodedInputStream cis = CodedInputStream.newInstance(buf.array(), offset, buf.limit());
       int headerSize = cis.readRawVarint32();
       offset = cis.getTotalBytesRead();
       Message.Builder builder = RequestHeader.newBuilder();
-      ProtobufUtil.mergeFrom(builder, buf, offset, headerSize);
+      ProtobufUtil.mergeFrom(builder, cis, headerSize);
       RequestHeader header = (RequestHeader) builder.build();
       offset += headerSize;
       int id = header.getCallId();
@@ -1862,19 +1873,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
           md = this.service.getDescriptorForType().findMethodByName(header.getMethodName());
           if (md == null) throw new UnsupportedOperationException(header.getMethodName());
           builder = this.service.getRequestPrototype(md).newBuilderForType();
-          // To read the varint, I need an inputstream; might as well be a CIS.
-          cis = CodedInputStream.newInstance(buf, offset, buf.length);
+          cis.resetSizeCounter();
           int paramSize = cis.readRawVarint32();
           offset += cis.getTotalBytesRead();
           if (builder != null) {
-            ProtobufUtil.mergeFrom(builder, buf, offset, paramSize);
+            ProtobufUtil.mergeFrom(builder, cis, paramSize);
             param = builder.build();
           }
           offset += paramSize;
         }
         if (header.hasCellBlockMeta()) {
-          cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec,
-            buf, offset, buf.length);
+          buf.position(offset);
+          cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec, buf);
         }
       } catch (Throwable t) {
         InetSocketAddress address = getListenerAddress();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
index cfdbce0..8438378 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoReque
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
@@ -217,10 +216,10 @@ public class AnnotationReadingPriorityFunction implements PriorityFunction {
     if (param == null) {
       return HConstants.NORMAL_QOS;
     }
-    if (param instanceof MultiRequest) {
-      // The multi call has its priority set in the header.  All calls should work this way but
-      // only this one has been converted so far.  No priority == NORMAL_QOS.
-      return header.hasPriority()? header.getPriority(): HConstants.NORMAL_QOS;
+
+    // Trust the client-set priorities if set
+    if (header.hasPriority()) {
+      return header.getPriority();
     }
 
     String cls = param.getClass().getName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/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 3cf4122..99b571f 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
@@ -5177,6 +5177,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @param readLock is the lock reader or writer. True indicates that a non-exlcusive
    *                 lock is requested
    */
+  @Override
   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
     // Make sure the row is inside of this region before getting the lock for it.
     checkRow(row, "row lock");
@@ -5592,8 +5593,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       // Here we separate all scanners into two lists - scanner that provide data required
       // by the filter to operate (scanners list) and all others (joinedScanners list).
-      List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
-      List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
+      List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
+      List<KeyValueScanner> joinedScanners
+        = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
       if (additionalScanners != null) {
         scanners.addAll(additionalScanners);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a69272ef/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 7eaadc2..3e133c4 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
@@ -1147,8 +1147,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   Region getRegion(
       final RegionSpecifier regionSpecifier) throws IOException {
-    return regionServer.getRegionByEncodedName(regionSpecifier.getValue().toByteArray(),
-        ProtobufUtil.getRegionEncodedName(regionSpecifier));
+    ByteString value = regionSpecifier.getValue();
+    RegionSpecifierType type = regionSpecifier.getType();
+    switch (type) {
+      case REGION_NAME:
+        byte[] regionName = value.toByteArray();
+        String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
+        return regionServer.getRegionByEncodedName(regionName, encodedRegionName);
+      case ENCODED_REGION_NAME:
+        return regionServer.getRegionByEncodedName(value.toStringUtf8());
+      default:
+        throw new DoNotRetryIOException(
+          "Unsupported region specifier type: " + type);
+    }
   }
 
   @VisibleForTesting


[16/32] hbase git commit: Fixup on the hbasecon banner image.. add date and location -- git add image

Posted by sy...@apache.org.
Fixup on the hbasecon banner image.. add date and location -- git add image


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

Branch: refs/heads/hbase-12439
Commit: 9c832109458ced70d0db701e7f58ef41b1df0fc7
Parents: 64bac77
Author: stack <st...@apache.org>
Authored: Fri Feb 5 09:01:04 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 09:01:16 2016 -0800

----------------------------------------------------------------------
 .../site/resources/images/hbasecon2016-stacked.png | Bin 0 -> 24924 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


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


[03/32] hbase git commit: HBASE-15211 Don't run the CatalogJanitor if there are regions in transition

Posted by sy...@apache.org.
HBASE-15211 Don't run the CatalogJanitor if there are regions in transition


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

Branch: refs/heads/hbase-12439
Commit: f5fba2ba0d5ed9717d3a6439a24235710ac0886b
Parents: 2cf8af5
Author: Elliott Clark <ec...@apache.org>
Authored: Wed Feb 3 13:38:53 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Feb 4 08:52:09 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/CatalogJanitor.java    | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f5fba2ba/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index d02e5ae..b9abc65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -96,7 +96,11 @@ public class CatalogJanitor extends ScheduledChore {
   @Override
   protected void chore() {
     try {
-      if (this.enabled.get()) {
+      AssignmentManager am = this.services.getAssignmentManager();
+      if (this.enabled.get()
+          && am != null
+          && am.isFailoverCleanupDone()
+          && am.getRegionStates().getRegionsInTransition().size() == 0) {
         scan();
       } else {
         LOG.warn("CatalogJanitor disabled! Not running scan.");


[15/32] hbase git commit: HBASE-11262 Avoid empty columns while doing bulk-load (Ashish Kumar)

Posted by sy...@apache.org.
HBASE-11262 Avoid empty columns while doing bulk-load (Ashish Kumar)


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

Branch: refs/heads/hbase-12439
Commit: 64bac770d4cd3cb780fea58101ea79b96283d320
Parents: 6f6a8ed
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 5 09:00:04 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 5 09:00:04 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/ImportTsv.java   |  2 ++
 .../hadoop/hbase/mapreduce/TsvImporterMapper.java      |  7 ++++++-
 .../apache/hadoop/hbase/mapreduce/TestImportTsv.java   | 13 +++++++++++++
 3 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/64bac770/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index 96ab43b..e778d99 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -91,6 +91,7 @@ public class ImportTsv extends Configured implements Tool {
   // If true, bad lines are logged to stderr. Default: false.
   public final static String LOG_BAD_LINES_CONF_KEY = "importtsv.log.bad.lines";
   public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
+  public final static String SKIP_EMPTY_COLUMNS = "importtsv.skip.empty.columns";
   public final static String COLUMNS_CONF_KEY = "importtsv.columns";
   public final static String SEPARATOR_CONF_KEY = "importtsv.separator";
   public final static String ATTRIBUTE_SEPERATOR_CONF_KEY = "attributes.seperator";
@@ -685,6 +686,7 @@ public class ImportTsv extends Configured implements Tool {
       " table. If table does not exist, it is created but deleted in the end.\n" +
       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
       "  -D" + LOG_BAD_LINES_CONF_KEY + "=true - logs invalid lines to stderr\n" +
+      "  -D" + SKIP_EMPTY_COLUMNS + "=false - If true then skip empty columns in bulk import\n" +
       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
       "  -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
       "  -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +

http://git-wip-us.apache.org/repos/asf/hbase/blob/64bac770/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
index e14874b..94bcb43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TsvImporterMapper.java
@@ -58,6 +58,8 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
 
   /** Should skip bad lines */
   private boolean skipBadLines;
+  /** Should skip empty columns*/
+  private boolean skipEmptyColumns;
   private Counter badLineCount;
   private boolean logBadLines;
 
@@ -133,6 +135,8 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
     // configuration.
     ts = conf.getLong(ImportTsv.TIMESTAMP_CONF_KEY, 0);
 
+    skipEmptyColumns = context.getConfiguration().getBoolean(
+        ImportTsv.SKIP_EMPTY_COLUMNS, false);
     skipBadLines = context.getConfiguration().getBoolean(
         ImportTsv.SKIP_LINES_CONF_KEY, true);
     badLineCount = context.getCounter("ImportTsv", "Bad Lines");
@@ -178,7 +182,8 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
       for (int i = 0; i < parsed.getColumnCount(); i++) {
         if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()
             || i == parser.getAttributesKeyColumnIndex() || i == parser.getCellVisibilityColumnIndex()
-            || i == parser.getCellTTLColumnIndex()) {
+            || i == parser.getCellTTLColumnIndex() || (skipEmptyColumns 
+            && parsed.getColumnLength(i) == 0)) {
           continue;
         }
         populatePut(lineBytes, parsed, put, i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/64bac770/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
index 9b7c4ae..fd51544 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
@@ -363,6 +363,19 @@ public class TestImportTsv implements Configurable {
     doMROnTableTest(util, tn, FAMILY, data, args, 1, 4);
     util.deleteTable(tn);
   }
+  
+  @Test
+  public void testSkipEmptyColumns() throws Exception {
+    Path bulkOutputPath = new Path(util.getDataTestDirOnTestFS(tn.getNameAsString()), "hfiles");
+    args.put(ImportTsv.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
+    args.put(ImportTsv.COLUMNS_CONF_KEY, "HBASE_ROW_KEY,HBASE_TS_KEY,FAM:A,FAM:B");
+    args.put(ImportTsv.SEPARATOR_CONF_KEY, ",");
+    args.put(ImportTsv.SKIP_EMPTY_COLUMNS, "true");
+    // 2 Rows of data as input. Both rows are valid and only 3 columns are no-empty among 4
+    String data = "KEY,1234,VALUE1,VALUE2\nKEY,1235,,VALUE2\n";
+    doMROnTableTest(util, tn, FAMILY, data, args, 1, 3);
+    util.deleteTable(tn);
+  }
 
   private Tool doMROnTableTest(String data, int valueMultiplier) throws Exception {
     return doMROnTableTest(util, tn, FAMILY, data, args, valueMultiplier,-1);


[23/32] hbase git commit: HBASE-15122 Servlets generate XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER findbugs warnings (Samir Ahmic)

Posted by sy...@apache.org.
HBASE-15122 Servlets generate XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER findbugs warnings (Samir Ahmic)


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

Branch: refs/heads/hbase-12439
Commit: efc7a0d34749091d8efa623e7424956b72d3bb59
Parents: 2ce31f8
Author: stack <st...@apache.org>
Authored: Fri Feb 5 21:21:18 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 21:28:46 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/supplemental-models.xml  |  36 ++
 hbase-server/pom.xml                            |  11 +
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |   8 +-
 .../src/main/resources/ESAPI.properties         | 431 +++++++++++++++++++
 .../hbase/http/jmx/TestJMXJsonServlet.java      |   6 +
 .../src/test/resources/ESAPI.properties         | 431 +++++++++++++++++++
 pom.xml                                         |   1 +
 7 files changed, 923 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-resource-bundle/src/main/resources/supplemental-models.xml
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/supplemental-models.xml b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
index 2f94226..764667c 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -61,6 +61,24 @@ under the License.
       </licenses>
     </project>
   </supplement>
+  <supplement>
+    <project>
+      <groupId>commons-beanutils</groupId>
+      <artifactId>commons-beanutils-core</artifactId>
+
+      <organization>
+        <name>The Apache Software Foundation</name>
+        <url>http://www.apache.org/</url>
+      </organization>
+      <licenses>
+        <license>
+          <name>Apache Software License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
 <!-- Artifacts with ambiguously named licenses in POM -->
   <supplement>
     <project>
@@ -1195,4 +1213,22 @@ Copyright (c) 2007-2011 The JRuby project
       </licenses>
     </project>
   </supplement>
+  <supplement>
+    <project>
+      <groupId>xalan</groupId>
+      <artifactId>xalan</artifactId>
+
+      <organization>
+        <name>The Apache Software Foundation</name>
+        <url>http://www.apache.org/</url>
+      </organization>
+      <licenses>
+        <license>
+          <name>The Apache Software License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
 </supplementalDataModels>

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 3c25094..d5f1e30 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -561,6 +561,17 @@
       <artifactId>bcprov-jdk16</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.owasp.esapi</groupId>
+      <artifactId>esapi</artifactId>
+      <version>2.1.0</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>xercesImpl</artifactId>
+          <groupId>xerces</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
   <profiles>
     <!-- Needs to make the profile in apache parent pom -->

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index 45c2c15..14a19f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -35,6 +35,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.http.HttpServer;
 import org.apache.hadoop.hbase.util.JSONBean;
+import org.owasp.esapi.ESAPI;
 
 /*
  * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
@@ -167,7 +168,7 @@ public class JMXJsonServlet extends HttpServlet {
         jsonpcb = request.getParameter(CALLBACK_PARAM);
         if (jsonpcb != null) {
           response.setContentType("application/javascript; charset=utf8");
-          writer.write(jsonpcb + "(");
+          writer.write(encodeJS(jsonpcb) + "(");
         } else {
           response.setContentType("application/json; charset=utf8");
         }
@@ -220,4 +221,9 @@ public class JMXJsonServlet extends HttpServlet {
       response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
     }
   }
+
+  private String encodeJS(String inputStr) {
+    return ESAPI.encoder().encodeForJavaScript(inputStr);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-server/src/main/resources/ESAPI.properties
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/ESAPI.properties b/hbase-server/src/main/resources/ESAPI.properties
new file mode 100644
index 0000000..9074001
--- /dev/null
+++ b/hbase-server/src/main/resources/ESAPI.properties
@@ -0,0 +1,431 @@
+#
+# OWASP Enterprise Security API (ESAPI) Properties file -- PRODUCTION Version
+#
+# This file is part of the Open Web Application Security Project (OWASP)
+# Enterprise Security API (ESAPI) project. For details, please see
+# http://www.owasp.org/index.php/ESAPI.
+#
+# Copyright (c) 2008,2009 - The OWASP Foundation
+#
+# DISCUSS: This may cause a major backwards compatibility issue, etc. but
+#           from a name space perspective, we probably should have prefaced
+#           all the property names with ESAPI or at least OWASP. Otherwise
+#           there could be problems is someone loads this properties file into
+#           the System properties.  We could also put this file into the
+#           esapi.jar file (perhaps as a ResourceBundle) and then allow an external
+#           ESAPI properties be defined that would overwrite these defaults.
+#           That keeps the application's properties relatively simple as usually
+#           they will only want to override a few properties. If looks like we
+#           already support multiple override levels of this in the
+#           DefaultSecurityConfiguration class, but I'm suggesting placing the
+#           defaults in the esapi.jar itself. That way, if the jar is signed,
+#           we could detect if those properties had been tampered with. (The
+#           code to check the jar signatures is pretty simple... maybe 70-90 LOC,
+#           but off course there is an execution penalty (similar to the way
+#           that the separate sunjce.jar used to be when a class from it was
+#           first loaded). Thoughts?
+###############################################################################
+#
+# WARNING: Operating system protection should be used to lock down the .esapi
+# resources directory and all the files inside and all the directories all the
+# way up to the root directory of the file system.  Note that if you are using
+# file-based implementations, that some files may need to be read-write as they
+# get updated dynamically.
+#
+# Before using, be sure to update the MasterKey and MasterSalt as described below.
+# N.B.: If you had stored data that you have previously encrypted with ESAPI 1.4,
+#        you *must* FIRST decrypt it using ESAPI 1.4 and then (if so desired)
+#        re-encrypt it with ESAPI 2.0. If you fail to do this, you will NOT be
+#        able to decrypt your data with ESAPI 2.0.
+#
+#        YOU HAVE BEEN WARNED!!! More details are in the ESAPI 2.0 Release Notes.
+#
+#===========================================================================
+# ESAPI Configuration
+#
+# If true, then print all the ESAPI properties set here when they are loaded.
+# If false, they are not printed. Useful to reduce output when running JUnit tests.
+# If you need to troubleshoot a properties related problem, turning this on may help.
+# This is 'false' in the src/test/resources/.esapi version. It is 'true' by
+# default for reasons of backward compatibility with earlier ESAPI versions.
+ESAPI.printProperties=true
+
+# ESAPI is designed to be easily extensible. You can use the reference implementation
+# or implement your own providers to take advantage of your enterprise's security
+# infrastructure. The functions in ESAPI are referenced using the ESAPI locator, like:
+#
+#    String ciphertext =
+#        ESAPI.encryptor().encrypt("Secret message");   // Deprecated in 2.0
+#    CipherText cipherText =
+#        ESAPI.encryptor().encrypt(new PlainText("Secret message")); // Preferred
+#
+# Below you can specify the classname for the provider that you wish to use in your
+# application. The only requirement is that it implement the appropriate ESAPI interface.
+# This allows you to switch security implementations in the future without rewriting the
+# entire application.
+#
+# ExperimentalAccessController requires ESAPI-AccessControlPolicy.xml in .esapi directory
+ESAPI.AccessControl=org.owasp.esapi.reference.DefaultAccessController
+# FileBasedAuthenticator requires users.txt file in .esapi directory
+ESAPI.Authenticator=org.owasp.esapi.reference.FileBasedAuthenticator
+ESAPI.Encoder=org.owasp.esapi.reference.DefaultEncoder
+ESAPI.Encryptor=org.owasp.esapi.reference.crypto.JavaEncryptor
+
+ESAPI.Executor=org.owasp.esapi.reference.DefaultExecutor
+ESAPI.HTTPUtilities=org.owasp.esapi.reference.DefaultHTTPUtilities
+ESAPI.IntrusionDetector=org.owasp.esapi.reference.DefaultIntrusionDetector
+# Log4JFactory Requires log4j.xml or log4j.properties in classpath - http://www.laliluna.de/log4j-tutorial.html
+ESAPI.Logger=org.owasp.esapi.reference.Log4JLogFactory
+#ESAPI.Logger=org.owasp.esapi.reference.JavaLogFactory
+ESAPI.Randomizer=org.owasp.esapi.reference.DefaultRandomizer
+ESAPI.Validator=org.owasp.esapi.reference.DefaultValidator
+
+#===========================================================================
+# ESAPI Authenticator
+#
+Authenticator.AllowedLoginAttempts=3
+Authenticator.MaxOldPasswordHashes=13
+Authenticator.UsernameParameterName=username
+Authenticator.PasswordParameterName=password
+# RememberTokenDuration (in days)
+Authenticator.RememberTokenDuration=14
+# Session Timeouts (in minutes)
+Authenticator.IdleTimeoutDuration=20
+Authenticator.AbsoluteTimeoutDuration=120
+
+#===========================================================================
+# ESAPI Encoder
+#
+# ESAPI canonicalizes input before validation to prevent bypassing filters with encoded attacks.
+# Failure to canonicalize input is a very common mistake when implementing validation schemes.
+# Canonicalization is automatic when using the ESAPI Validator, but you can also use the
+# following code to canonicalize data.
+#
+#      ESAPI.Encoder().canonicalize( "%22hello world&#x22;" );
+#
+# Multiple encoding is when a single encoding format is applied multiple times, multiple
+# different encoding formats are applied, or when multiple formats are nested. Allowing
+# multiple encoding is strongly discouraged.
+Encoder.AllowMultipleEncoding=false
+#
+# The default list of codecs to apply when canonicalizing untrusted data. The list should include the codecs
+# for all downstream interpreters or decoders. For example, if the data is likely to end up in a URL, HTML, or
+# inside JavaScript, then the list of codecs below is appropriate. The order of the list is not terribly important.
+Encoder.DefaultCodecList=HTMLEntityCodec,PercentCodec,JavaScriptCodec
+
+
+#===========================================================================
+# ESAPI Encryption
+#
+# The ESAPI Encryptor provides basic cryptographic functions with a simplified API.
+# To get started, generate a new key using java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
+# There is not currently any support for key rotation, so be careful when changing your key and salt as it
+# will invalidate all signed, encrypted, and hashed data.
+#
+# WARNING: Not all combinations of algorithms and key lengths are supported.
+# If you choose to use a key length greater than 128, you MUST download the
+# unlimited strength policy files and install in the lib directory of your JRE/JDK.
+# See http://java.sun.com/javase/downloads/index.jsp for more information.
+#
+# Backward compatibility with ESAPI Java 1.4 is supported by the two deprecated API
+# methods, Encryptor.encrypt(String) and Encryptor.decrypt(String). However, whenever
+# possible, these methods should be avoided as they use ECB cipher mode, which in almost
+# all circumstances a poor choice because of it's weakness. CBC cipher mode is the default
+# for the new Encryptor encrypt / decrypt methods for ESAPI Java 2.0.  In general, you
+# should only use this compatibility setting if you have persistent data encrypted with
+# version 1.4 and even then, you should ONLY set this compatibility mode UNTIL
+# you have decrypted all of your old encrypted data and then re-encrypted it with
+# ESAPI 2.0 using CBC mode. If you have some reason to mix the deprecated 1.4 mode
+# with the new 2.0 methods, make sure that you use the same cipher algorithm for both
+# (256-bit AES was the default for 1.4; 128-bit is the default for 2.0; see below for
+# more details.) Otherwise, you will have to use the new 2.0 encrypt / decrypt methods
+# where you can specify a SecretKey. (Note that if you are using the 256-bit AES,
+# that requires downloading the special jurisdiction policy files mentioned above.)
+#
+#        ***** IMPORTANT: Do NOT forget to replace these with your own values! *****
+# To calculate these values, you can run:
+#        java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
+#
+Encryptor.MasterKey=
+Encryptor.MasterSalt=
+
+# Provides the default JCE provider that ESAPI will "prefer" for its symmetric
+# encryption and hashing. (That is it will look to this provider first, but it
+# will defer to other providers if the requested algorithm is not implemented
+# by this provider.) If left unset, ESAPI will just use your Java VM's current
+# preferred JCE provider, which is generally set in the file
+# "$JAVA_HOME/jre/lib/security/java.security".
+#
+# The main intent of this is to allow ESAPI symmetric encryption to be
+# used with a FIPS 140-2 compliant crypto-module. For details, see the section
+# "Using ESAPI Symmetric Encryption with FIPS 140-2 Cryptographic Modules" in
+# the ESAPI 2.0 Symmetric Encryption User Guide, at:
+# http://owasp-esapi-java.googlecode.com/svn/trunk/documentation/esapi4java-core-2.0-symmetric-crypto-user-guide.html
+# However, this property also allows you to easily use an alternate JCE provider
+# such as "Bouncy Castle" without having to make changes to "java.security".
+# See Javadoc for SecurityProviderLoader for further details. If you wish to use
+# a provider that is not known to SecurityProviderLoader, you may specify the
+# fully-qualified class name of the JCE provider class that implements
+# java.security.Provider. If the name contains a '.', this is interpreted as
+# a fully-qualified class name that implements java.security.Provider.
+#
+# NOTE: Setting this property has the side-effect of changing it in your application
+#       as well, so if you are using JCE in your application directly rather than
+#       through ESAPI (you wouldn't do that, would you? ;-), it will change the
+#       preferred JCE provider there as well.
+#
+# Default: Keeps the JCE provider set to whatever JVM sets it to.
+Encryptor.PreferredJCEProvider=
+
+# AES is the most widely used and strongest encryption algorithm. This
+# should agree with your Encryptor.CipherTransformation property.
+# By default, ESAPI Java 1.4 uses "PBEWithMD5AndDES" and which is
+# very weak. It is essentially a password-based encryption key, hashed
+# with MD5 around 1K times and then encrypted with the weak DES algorithm
+# (56-bits) using ECB mode and an unspecified padding (it is
+# JCE provider specific, but most likely "NoPadding"). However, 2.0 uses
+# "AES/CBC/PKCSPadding". If you want to change these, change them here.
+# Warning: This property does not control the default reference implementation for
+#           ESAPI 2.0 using JavaEncryptor. Also, this property will be dropped
+#           in the future.
+# @deprecated
+Encryptor.EncryptionAlgorithm=AES
+#        For ESAPI Java 2.0 - New encrypt / decrypt methods use this.
+Encryptor.CipherTransformation=AES/CBC/PKCS5Padding
+
+# Applies to ESAPI 2.0 and later only!
+# Comma-separated list of cipher modes that provide *BOTH*
+# confidentiality *AND* message authenticity. (NIST refers to such cipher
+# modes as "combined modes" so that's what we shall call them.) If any of these
+# cipher modes are used then no MAC is calculated and stored
+# in the CipherText upon encryption. Likewise, if one of these
+# cipher modes is used with decryption, no attempt will be made
+# to validate the MAC contained in the CipherText object regardless
+# of whether it contains one or not. Since the expectation is that
+# these cipher modes support support message authenticity already,
+# injecting a MAC in the CipherText object would be at best redundant.
+#
+# Note that as of JDK 1.5, the SunJCE provider does not support *any*
+# of these cipher modes. Of these listed, only GCM and CCM are currently
+# NIST approved. YMMV for other JCE providers. E.g., Bouncy Castle supports
+# GCM and CCM with "NoPadding" mode, but not with "PKCS5Padding" or other
+# padding modes.
+Encryptor.cipher_modes.combined_modes=GCM,CCM,IAPM,EAX,OCB,CWC
+
+# Applies to ESAPI 2.0 and later only!
+# Additional cipher modes allowed for ESAPI 2.0 encryption. These
+# cipher modes are in _addition_ to those specified by the property
+# 'Encryptor.cipher_modes.combined_modes'.
+# Note: We will add support for streaming modes like CFB & OFB once
+# we add support for 'specified' to the property 'Encryptor.ChooseIVMethod'
+# (probably in ESAPI 2.1).
+# DISCUSS: Better name?
+Encryptor.cipher_modes.additional_allowed=CBC
+
+# 128-bit is almost always sufficient and appears to be more resistant to
+# related key attacks than is 256-bit AES. Use '_' to use default key size
+# for cipher algorithms (where it makes sense because the algorithm supports
+# a variable key size). Key length must agree to what's provided as the
+# cipher transformation, otherwise this will be ignored after logging a
+# warning.
+#
+# NOTE: This is what applies BOTH ESAPI 1.4 and 2.0. See warning above about mixing!
+Encryptor.EncryptionKeyLength=128
+
+# Because 2.0 uses CBC mode by default, it requires an initialization vector (IV).
+# (All cipher modes except ECB require an IV.) There are two choices: we can either
+# use a fixed IV known to both parties or allow ESAPI to choose a random IV. While
+# the IV does not need to be hidden from adversaries, it is important that the
+# adversary not be allowed to choose it. Also, random IVs are generally much more
+# secure than fixed IVs. (In fact, it is essential that feed-back cipher modes
+# such as CFB and OFB use a different IV for each encryption with a given key so
+# in such cases, random IVs are much preferred. By default, ESAPI 2.0 uses random
+# IVs. If you wish to use 'fixed' IVs, set 'Encryptor.ChooseIVMethod=fixed' and
+# uncomment the Encryptor.fixedIV.
+#
+# Valid values:        random|fixed|specified        'specified' not yet implemented; planned for 2.1
+Encryptor.ChooseIVMethod=random
+# If you choose to use a fixed IV, then you must place a fixed IV here that
+# is known to all others who are sharing your secret key. The format should
+# be a hex string that is the same length as the cipher block size for the
+# cipher algorithm that you are using. The following is an example for AES
+# from an AES test vector for AES-128/CBC as described in:
+# NIST Special Publication 800-38A (2001 Edition)
+# "Recommendation for Block Cipher Modes of Operation".
+# (Note that the block size for AES is 16 bytes == 128 bits.)
+#
+Encryptor.fixedIV=0x000102030405060708090a0b0c0d0e0f
+
+# Whether or not CipherText should use a message authentication code (MAC) with it.
+# This prevents an adversary from altering the IV as well as allowing a more
+# fool-proof way of determining the decryption failed because of an incorrect
+# key being supplied. This refers to the "separate" MAC calculated and stored
+# in CipherText, not part of any MAC that is calculated as a result of a
+# "combined mode" cipher mode.
+#
+# If you are using ESAPI with a FIPS 140-2 cryptographic module, you *must* also
+# set this property to false.
+Encryptor.CipherText.useMAC=true
+
+# Whether or not the PlainText object may be overwritten and then marked
+# eligible for garbage collection. If not set, this is still treated as 'true'.
+Encryptor.PlainText.overwrite=true
+
+# Do not use DES except in a legacy situations. 56-bit is way too small key size.
+#Encryptor.EncryptionKeyLength=56
+#Encryptor.EncryptionAlgorithm=DES
+
+# TripleDES is considered strong enough for most purposes.
+#    Note:    There is also a 112-bit version of DESede. Using the 168-bit version
+#            requires downloading the special jurisdiction policy from Sun.
+#Encryptor.EncryptionKeyLength=168
+#Encryptor.EncryptionAlgorithm=DESede
+
+Encryptor.HashAlgorithm=SHA-512
+Encryptor.HashIterations=1024
+Encryptor.DigitalSignatureAlgorithm=SHA1withDSA
+Encryptor.DigitalSignatureKeyLength=1024
+Encryptor.RandomAlgorithm=SHA1PRNG
+Encryptor.CharacterEncoding=UTF-8
+
+
+#===========================================================================
+# ESAPI HttpUtilties
+#
+# The HttpUtilities provide basic protections to HTTP requests and responses. Primarily these methods
+# protect against malicious data from attackers, such as unprintable characters, escaped characters,
+# and other simple attacks. The HttpUtilities also provides utility methods for dealing with cookies,
+# headers, and CSRF tokens.
+#
+# Default file upload location (remember to escape backslashes with \\)
+HttpUtilities.UploadDir=C:\\ESAPI\\testUpload
+HttpUtilities.UploadTempDir=C:\\temp
+# Force flags on cookies, if you use HttpUtilities to set cookies
+HttpUtilities.ForceHttpOnlySession=false
+HttpUtilities.ForceSecureSession=false
+HttpUtilities.ForceHttpOnlyCookies=true
+HttpUtilities.ForceSecureCookies=true
+# Maximum size of HTTP headers
+HttpUtilities.MaxHeaderSize=4096
+# File upload configuration
+HttpUtilities.ApprovedUploadExtensions=.zip,.pdf,.doc,.docx,.ppt,.pptx,.tar,.gz,.tgz,.rar,.war,.jar,.ear,.xls,.rtf,.properties,.java,.class,.txt,.xml,.jsp,.jsf,.exe,.dll
+HttpUtilities.MaxUploadFileBytes=500000000
+# Using UTF-8 throughout your stack is highly recommended. That includes your database driver,
+# container, and any other technologies you may be using. Failure to do this may expose you
+# to Unicode transcoding injection attacks. Use of UTF-8 does not hinder internationalization.
+HttpUtilities.ResponseContentType=text/html; charset=UTF-8
+
+
+
+#===========================================================================
+# ESAPI Executor
+# CHECKME - Not sure what this is used for, but surely it should be made OS independent.
+Executor.WorkingDirectory=C:\\Windows\\Temp
+Executor.ApprovedExecutables=C:\\Windows\\System32\\cmd.exe,C:\\Windows\\System32\\runas.exe
+
+
+#===========================================================================
+# ESAPI Logging
+# Set the application name if these logs are combined with other applications
+Logger.ApplicationName=ExampleApplication
+# If you use an HTML log viewer that does not properly HTML escape log data, you can set LogEncodingRequired to true
+Logger.LogEncodingRequired=false
+# Determines whether ESAPI should log the application name. This might be clutter in some single-server/single-app environments.
+Logger.LogApplicationName=true
+# Determines whether ESAPI should log the server IP and port. This might be clutter in some single-server environments.
+Logger.LogServerIP=true
+# LogFileName, the name of the logging file. Provide a full directory path (e.g., C:\\ESAPI\\ESAPI_logging_file) if you
+# want to place it in a specific directory.
+Logger.LogFileName=ESAPI_logging_file
+# MaxLogFileSize, the max size (in bytes) of a single log file before it cuts over to a new one (default is 10,000,000)
+Logger.MaxLogFileSize=10000000
+
+
+#===========================================================================
+# ESAPI Intrusion Detection
+#
+# Each event has a base to which .count, .interval, and .action are added
+# The IntrusionException will fire if we receive "count" events within "interval" seconds
+# The IntrusionDetector is configurable to take the following actions: log, logout, and disable
+#  (multiple actions separated by commas are allowed e.g. event.test.actions=log,disable
+#
+# Custom Events
+# Names must start with "event." as the base
+# Use IntrusionDetector.addEvent( "test" ) in your code to trigger "event.test" here
+# You can also disable intrusion detection completely by changing
+# the following parameter to true
+#
+IntrusionDetector.Disable=false
+#
+IntrusionDetector.event.test.count=2
+IntrusionDetector.event.test.interval=10
+IntrusionDetector.event.test.actions=disable,log
+
+# Exception Events
+# All EnterpriseSecurityExceptions are registered automatically
+# Call IntrusionDetector.getInstance().addException(e) for Exceptions that do not extend EnterpriseSecurityException
+# Use the fully qualified classname of the exception as the base
+
+# any intrusion is an attack
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.count=1
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.interval=1
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.actions=log,disable,logout
+
+# for test purposes
+# CHECKME: Shouldn't there be something in the property name itself that designates
+#           that these are for testing???
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.count=10
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.interval=5
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.actions=log,disable,logout
+
+# rapid validation errors indicate scans or attacks in progress
+# org.owasp.esapi.errors.ValidationException.count=10
+# org.owasp.esapi.errors.ValidationException.interval=10
+# org.owasp.esapi.errors.ValidationException.actions=log,logout
+
+# sessions jumping between hosts indicates session hijacking
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.count=2
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.interval=10
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.actions=log,logout
+
+
+#===========================================================================
+# ESAPI Validation
+#
+# The ESAPI Validator works on regular expressions with defined names. You can define names
+# either here, or you may define application specific patterns in a separate file defined below.
+# This allows enterprises to specify both organizational standards as well as application specific
+# validation rules.
+#
+Validator.ConfigurationFile=validation.properties
+
+# Validators used by ESAPI
+Validator.AccountName=^[a-zA-Z0-9]{3,20}$
+Validator.SystemCommand=^[a-zA-Z\\-\\/]{1,64}$
+Validator.RoleName=^[a-z]{1,20}$
+
+#the word TEST below should be changed to your application
+#name - only relative URL's are supported
+Validator.Redirect=^\\/test.*$
+
+# Global HTTP Validation Rules
+# Values with Base64 encoded data (e.g. encrypted state) will need at least [a-zA-Z0-9\/+=]
+Validator.HTTPScheme=^(http|https)$
+Validator.HTTPServerName=^[a-zA-Z0-9_.\\-]*$
+Validator.HTTPParameterName=^[a-zA-Z0-9_]{1,32}$
+Validator.HTTPParameterValue=^[a-zA-Z0-9.\\-\\/+=_ ]*$
+Validator.HTTPCookieName=^[a-zA-Z0-9\\-_]{1,32}$
+Validator.HTTPCookieValue=^[a-zA-Z0-9\\-\\/+=_ ]*$
+Validator.HTTPHeaderName=^[a-zA-Z0-9\\-_]{1,32}$
+Validator.HTTPHeaderValue=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
+Validator.HTTPContextPath=^[a-zA-Z0-9.\\-\\/_]*$
+Validator.HTTPServletPath=^[a-zA-Z0-9.\\-\\/_]*$
+Validator.HTTPPath=^[a-zA-Z0-9.\\-_]*$
+Validator.HTTPQueryString=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ %]*$
+Validator.HTTPURI=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
+Validator.HTTPURL=^.*$
+Validator.HTTPJSESSIONID=^[A-Z0-9]{10,30}$
+
+# Validation of file related input
+Validator.FileName=^[a-zA-Z0-9!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$
+Validator.DirectoryName=^[a-zA-Z0-9:/\\\\!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
index 031ddce..baeaf89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
@@ -105,5 +105,11 @@ public class TestJMXJsonServlet extends HttpServerFunctionalTest {
     assertReFind("\"committed\"\\s*:", result);
     assertReFind("\\}\\);$", result);
 
+    // test to get XSS JSONP result
+    result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Memory&callback=<script>alert('hello')</script>"));
+    LOG.info("/jmx?qry=java.lang:type=Memory&callback=<script>alert('hello')</script> RESULT: "+result);
+    assertTrue(!result.contains("<script>"));
+
+
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/hbase-server/src/test/resources/ESAPI.properties
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/ESAPI.properties b/hbase-server/src/test/resources/ESAPI.properties
new file mode 100644
index 0000000..9074001
--- /dev/null
+++ b/hbase-server/src/test/resources/ESAPI.properties
@@ -0,0 +1,431 @@
+#
+# OWASP Enterprise Security API (ESAPI) Properties file -- PRODUCTION Version
+#
+# This file is part of the Open Web Application Security Project (OWASP)
+# Enterprise Security API (ESAPI) project. For details, please see
+# http://www.owasp.org/index.php/ESAPI.
+#
+# Copyright (c) 2008,2009 - The OWASP Foundation
+#
+# DISCUSS: This may cause a major backwards compatibility issue, etc. but
+#           from a name space perspective, we probably should have prefaced
+#           all the property names with ESAPI or at least OWASP. Otherwise
+#           there could be problems is someone loads this properties file into
+#           the System properties.  We could also put this file into the
+#           esapi.jar file (perhaps as a ResourceBundle) and then allow an external
+#           ESAPI properties be defined that would overwrite these defaults.
+#           That keeps the application's properties relatively simple as usually
+#           they will only want to override a few properties. If looks like we
+#           already support multiple override levels of this in the
+#           DefaultSecurityConfiguration class, but I'm suggesting placing the
+#           defaults in the esapi.jar itself. That way, if the jar is signed,
+#           we could detect if those properties had been tampered with. (The
+#           code to check the jar signatures is pretty simple... maybe 70-90 LOC,
+#           but off course there is an execution penalty (similar to the way
+#           that the separate sunjce.jar used to be when a class from it was
+#           first loaded). Thoughts?
+###############################################################################
+#
+# WARNING: Operating system protection should be used to lock down the .esapi
+# resources directory and all the files inside and all the directories all the
+# way up to the root directory of the file system.  Note that if you are using
+# file-based implementations, that some files may need to be read-write as they
+# get updated dynamically.
+#
+# Before using, be sure to update the MasterKey and MasterSalt as described below.
+# N.B.: If you had stored data that you have previously encrypted with ESAPI 1.4,
+#        you *must* FIRST decrypt it using ESAPI 1.4 and then (if so desired)
+#        re-encrypt it with ESAPI 2.0. If you fail to do this, you will NOT be
+#        able to decrypt your data with ESAPI 2.0.
+#
+#        YOU HAVE BEEN WARNED!!! More details are in the ESAPI 2.0 Release Notes.
+#
+#===========================================================================
+# ESAPI Configuration
+#
+# If true, then print all the ESAPI properties set here when they are loaded.
+# If false, they are not printed. Useful to reduce output when running JUnit tests.
+# If you need to troubleshoot a properties related problem, turning this on may help.
+# This is 'false' in the src/test/resources/.esapi version. It is 'true' by
+# default for reasons of backward compatibility with earlier ESAPI versions.
+ESAPI.printProperties=true
+
+# ESAPI is designed to be easily extensible. You can use the reference implementation
+# or implement your own providers to take advantage of your enterprise's security
+# infrastructure. The functions in ESAPI are referenced using the ESAPI locator, like:
+#
+#    String ciphertext =
+#        ESAPI.encryptor().encrypt("Secret message");   // Deprecated in 2.0
+#    CipherText cipherText =
+#        ESAPI.encryptor().encrypt(new PlainText("Secret message")); // Preferred
+#
+# Below you can specify the classname for the provider that you wish to use in your
+# application. The only requirement is that it implement the appropriate ESAPI interface.
+# This allows you to switch security implementations in the future without rewriting the
+# entire application.
+#
+# ExperimentalAccessController requires ESAPI-AccessControlPolicy.xml in .esapi directory
+ESAPI.AccessControl=org.owasp.esapi.reference.DefaultAccessController
+# FileBasedAuthenticator requires users.txt file in .esapi directory
+ESAPI.Authenticator=org.owasp.esapi.reference.FileBasedAuthenticator
+ESAPI.Encoder=org.owasp.esapi.reference.DefaultEncoder
+ESAPI.Encryptor=org.owasp.esapi.reference.crypto.JavaEncryptor
+
+ESAPI.Executor=org.owasp.esapi.reference.DefaultExecutor
+ESAPI.HTTPUtilities=org.owasp.esapi.reference.DefaultHTTPUtilities
+ESAPI.IntrusionDetector=org.owasp.esapi.reference.DefaultIntrusionDetector
+# Log4JFactory Requires log4j.xml or log4j.properties in classpath - http://www.laliluna.de/log4j-tutorial.html
+ESAPI.Logger=org.owasp.esapi.reference.Log4JLogFactory
+#ESAPI.Logger=org.owasp.esapi.reference.JavaLogFactory
+ESAPI.Randomizer=org.owasp.esapi.reference.DefaultRandomizer
+ESAPI.Validator=org.owasp.esapi.reference.DefaultValidator
+
+#===========================================================================
+# ESAPI Authenticator
+#
+Authenticator.AllowedLoginAttempts=3
+Authenticator.MaxOldPasswordHashes=13
+Authenticator.UsernameParameterName=username
+Authenticator.PasswordParameterName=password
+# RememberTokenDuration (in days)
+Authenticator.RememberTokenDuration=14
+# Session Timeouts (in minutes)
+Authenticator.IdleTimeoutDuration=20
+Authenticator.AbsoluteTimeoutDuration=120
+
+#===========================================================================
+# ESAPI Encoder
+#
+# ESAPI canonicalizes input before validation to prevent bypassing filters with encoded attacks.
+# Failure to canonicalize input is a very common mistake when implementing validation schemes.
+# Canonicalization is automatic when using the ESAPI Validator, but you can also use the
+# following code to canonicalize data.
+#
+#      ESAPI.Encoder().canonicalize( "%22hello world&#x22;" );
+#
+# Multiple encoding is when a single encoding format is applied multiple times, multiple
+# different encoding formats are applied, or when multiple formats are nested. Allowing
+# multiple encoding is strongly discouraged.
+Encoder.AllowMultipleEncoding=false
+#
+# The default list of codecs to apply when canonicalizing untrusted data. The list should include the codecs
+# for all downstream interpreters or decoders. For example, if the data is likely to end up in a URL, HTML, or
+# inside JavaScript, then the list of codecs below is appropriate. The order of the list is not terribly important.
+Encoder.DefaultCodecList=HTMLEntityCodec,PercentCodec,JavaScriptCodec
+
+
+#===========================================================================
+# ESAPI Encryption
+#
+# The ESAPI Encryptor provides basic cryptographic functions with a simplified API.
+# To get started, generate a new key using java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
+# There is not currently any support for key rotation, so be careful when changing your key and salt as it
+# will invalidate all signed, encrypted, and hashed data.
+#
+# WARNING: Not all combinations of algorithms and key lengths are supported.
+# If you choose to use a key length greater than 128, you MUST download the
+# unlimited strength policy files and install in the lib directory of your JRE/JDK.
+# See http://java.sun.com/javase/downloads/index.jsp for more information.
+#
+# Backward compatibility with ESAPI Java 1.4 is supported by the two deprecated API
+# methods, Encryptor.encrypt(String) and Encryptor.decrypt(String). However, whenever
+# possible, these methods should be avoided as they use ECB cipher mode, which in almost
+# all circumstances a poor choice because of it's weakness. CBC cipher mode is the default
+# for the new Encryptor encrypt / decrypt methods for ESAPI Java 2.0.  In general, you
+# should only use this compatibility setting if you have persistent data encrypted with
+# version 1.4 and even then, you should ONLY set this compatibility mode UNTIL
+# you have decrypted all of your old encrypted data and then re-encrypted it with
+# ESAPI 2.0 using CBC mode. If you have some reason to mix the deprecated 1.4 mode
+# with the new 2.0 methods, make sure that you use the same cipher algorithm for both
+# (256-bit AES was the default for 1.4; 128-bit is the default for 2.0; see below for
+# more details.) Otherwise, you will have to use the new 2.0 encrypt / decrypt methods
+# where you can specify a SecretKey. (Note that if you are using the 256-bit AES,
+# that requires downloading the special jurisdiction policy files mentioned above.)
+#
+#        ***** IMPORTANT: Do NOT forget to replace these with your own values! *****
+# To calculate these values, you can run:
+#        java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
+#
+Encryptor.MasterKey=
+Encryptor.MasterSalt=
+
+# Provides the default JCE provider that ESAPI will "prefer" for its symmetric
+# encryption and hashing. (That is it will look to this provider first, but it
+# will defer to other providers if the requested algorithm is not implemented
+# by this provider.) If left unset, ESAPI will just use your Java VM's current
+# preferred JCE provider, which is generally set in the file
+# "$JAVA_HOME/jre/lib/security/java.security".
+#
+# The main intent of this is to allow ESAPI symmetric encryption to be
+# used with a FIPS 140-2 compliant crypto-module. For details, see the section
+# "Using ESAPI Symmetric Encryption with FIPS 140-2 Cryptographic Modules" in
+# the ESAPI 2.0 Symmetric Encryption User Guide, at:
+# http://owasp-esapi-java.googlecode.com/svn/trunk/documentation/esapi4java-core-2.0-symmetric-crypto-user-guide.html
+# However, this property also allows you to easily use an alternate JCE provider
+# such as "Bouncy Castle" without having to make changes to "java.security".
+# See Javadoc for SecurityProviderLoader for further details. If you wish to use
+# a provider that is not known to SecurityProviderLoader, you may specify the
+# fully-qualified class name of the JCE provider class that implements
+# java.security.Provider. If the name contains a '.', this is interpreted as
+# a fully-qualified class name that implements java.security.Provider.
+#
+# NOTE: Setting this property has the side-effect of changing it in your application
+#       as well, so if you are using JCE in your application directly rather than
+#       through ESAPI (you wouldn't do that, would you? ;-), it will change the
+#       preferred JCE provider there as well.
+#
+# Default: Keeps the JCE provider set to whatever JVM sets it to.
+Encryptor.PreferredJCEProvider=
+
+# AES is the most widely used and strongest encryption algorithm. This
+# should agree with your Encryptor.CipherTransformation property.
+# By default, ESAPI Java 1.4 uses "PBEWithMD5AndDES" and which is
+# very weak. It is essentially a password-based encryption key, hashed
+# with MD5 around 1K times and then encrypted with the weak DES algorithm
+# (56-bits) using ECB mode and an unspecified padding (it is
+# JCE provider specific, but most likely "NoPadding"). However, 2.0 uses
+# "AES/CBC/PKCSPadding". If you want to change these, change them here.
+# Warning: This property does not control the default reference implementation for
+#           ESAPI 2.0 using JavaEncryptor. Also, this property will be dropped
+#           in the future.
+# @deprecated
+Encryptor.EncryptionAlgorithm=AES
+#        For ESAPI Java 2.0 - New encrypt / decrypt methods use this.
+Encryptor.CipherTransformation=AES/CBC/PKCS5Padding
+
+# Applies to ESAPI 2.0 and later only!
+# Comma-separated list of cipher modes that provide *BOTH*
+# confidentiality *AND* message authenticity. (NIST refers to such cipher
+# modes as "combined modes" so that's what we shall call them.) If any of these
+# cipher modes are used then no MAC is calculated and stored
+# in the CipherText upon encryption. Likewise, if one of these
+# cipher modes is used with decryption, no attempt will be made
+# to validate the MAC contained in the CipherText object regardless
+# of whether it contains one or not. Since the expectation is that
+# these cipher modes support support message authenticity already,
+# injecting a MAC in the CipherText object would be at best redundant.
+#
+# Note that as of JDK 1.5, the SunJCE provider does not support *any*
+# of these cipher modes. Of these listed, only GCM and CCM are currently
+# NIST approved. YMMV for other JCE providers. E.g., Bouncy Castle supports
+# GCM and CCM with "NoPadding" mode, but not with "PKCS5Padding" or other
+# padding modes.
+Encryptor.cipher_modes.combined_modes=GCM,CCM,IAPM,EAX,OCB,CWC
+
+# Applies to ESAPI 2.0 and later only!
+# Additional cipher modes allowed for ESAPI 2.0 encryption. These
+# cipher modes are in _addition_ to those specified by the property
+# 'Encryptor.cipher_modes.combined_modes'.
+# Note: We will add support for streaming modes like CFB & OFB once
+# we add support for 'specified' to the property 'Encryptor.ChooseIVMethod'
+# (probably in ESAPI 2.1).
+# DISCUSS: Better name?
+Encryptor.cipher_modes.additional_allowed=CBC
+
+# 128-bit is almost always sufficient and appears to be more resistant to
+# related key attacks than is 256-bit AES. Use '_' to use default key size
+# for cipher algorithms (where it makes sense because the algorithm supports
+# a variable key size). Key length must agree to what's provided as the
+# cipher transformation, otherwise this will be ignored after logging a
+# warning.
+#
+# NOTE: This is what applies BOTH ESAPI 1.4 and 2.0. See warning above about mixing!
+Encryptor.EncryptionKeyLength=128
+
+# Because 2.0 uses CBC mode by default, it requires an initialization vector (IV).
+# (All cipher modes except ECB require an IV.) There are two choices: we can either
+# use a fixed IV known to both parties or allow ESAPI to choose a random IV. While
+# the IV does not need to be hidden from adversaries, it is important that the
+# adversary not be allowed to choose it. Also, random IVs are generally much more
+# secure than fixed IVs. (In fact, it is essential that feed-back cipher modes
+# such as CFB and OFB use a different IV for each encryption with a given key so
+# in such cases, random IVs are much preferred. By default, ESAPI 2.0 uses random
+# IVs. If you wish to use 'fixed' IVs, set 'Encryptor.ChooseIVMethod=fixed' and
+# uncomment the Encryptor.fixedIV.
+#
+# Valid values:        random|fixed|specified        'specified' not yet implemented; planned for 2.1
+Encryptor.ChooseIVMethod=random
+# If you choose to use a fixed IV, then you must place a fixed IV here that
+# is known to all others who are sharing your secret key. The format should
+# be a hex string that is the same length as the cipher block size for the
+# cipher algorithm that you are using. The following is an example for AES
+# from an AES test vector for AES-128/CBC as described in:
+# NIST Special Publication 800-38A (2001 Edition)
+# "Recommendation for Block Cipher Modes of Operation".
+# (Note that the block size for AES is 16 bytes == 128 bits.)
+#
+Encryptor.fixedIV=0x000102030405060708090a0b0c0d0e0f
+
+# Whether or not CipherText should use a message authentication code (MAC) with it.
+# This prevents an adversary from altering the IV as well as allowing a more
+# fool-proof way of determining the decryption failed because of an incorrect
+# key being supplied. This refers to the "separate" MAC calculated and stored
+# in CipherText, not part of any MAC that is calculated as a result of a
+# "combined mode" cipher mode.
+#
+# If you are using ESAPI with a FIPS 140-2 cryptographic module, you *must* also
+# set this property to false.
+Encryptor.CipherText.useMAC=true
+
+# Whether or not the PlainText object may be overwritten and then marked
+# eligible for garbage collection. If not set, this is still treated as 'true'.
+Encryptor.PlainText.overwrite=true
+
+# Do not use DES except in a legacy situations. 56-bit is way too small key size.
+#Encryptor.EncryptionKeyLength=56
+#Encryptor.EncryptionAlgorithm=DES
+
+# TripleDES is considered strong enough for most purposes.
+#    Note:    There is also a 112-bit version of DESede. Using the 168-bit version
+#            requires downloading the special jurisdiction policy from Sun.
+#Encryptor.EncryptionKeyLength=168
+#Encryptor.EncryptionAlgorithm=DESede
+
+Encryptor.HashAlgorithm=SHA-512
+Encryptor.HashIterations=1024
+Encryptor.DigitalSignatureAlgorithm=SHA1withDSA
+Encryptor.DigitalSignatureKeyLength=1024
+Encryptor.RandomAlgorithm=SHA1PRNG
+Encryptor.CharacterEncoding=UTF-8
+
+
+#===========================================================================
+# ESAPI HttpUtilties
+#
+# The HttpUtilities provide basic protections to HTTP requests and responses. Primarily these methods
+# protect against malicious data from attackers, such as unprintable characters, escaped characters,
+# and other simple attacks. The HttpUtilities also provides utility methods for dealing with cookies,
+# headers, and CSRF tokens.
+#
+# Default file upload location (remember to escape backslashes with \\)
+HttpUtilities.UploadDir=C:\\ESAPI\\testUpload
+HttpUtilities.UploadTempDir=C:\\temp
+# Force flags on cookies, if you use HttpUtilities to set cookies
+HttpUtilities.ForceHttpOnlySession=false
+HttpUtilities.ForceSecureSession=false
+HttpUtilities.ForceHttpOnlyCookies=true
+HttpUtilities.ForceSecureCookies=true
+# Maximum size of HTTP headers
+HttpUtilities.MaxHeaderSize=4096
+# File upload configuration
+HttpUtilities.ApprovedUploadExtensions=.zip,.pdf,.doc,.docx,.ppt,.pptx,.tar,.gz,.tgz,.rar,.war,.jar,.ear,.xls,.rtf,.properties,.java,.class,.txt,.xml,.jsp,.jsf,.exe,.dll
+HttpUtilities.MaxUploadFileBytes=500000000
+# Using UTF-8 throughout your stack is highly recommended. That includes your database driver,
+# container, and any other technologies you may be using. Failure to do this may expose you
+# to Unicode transcoding injection attacks. Use of UTF-8 does not hinder internationalization.
+HttpUtilities.ResponseContentType=text/html; charset=UTF-8
+
+
+
+#===========================================================================
+# ESAPI Executor
+# CHECKME - Not sure what this is used for, but surely it should be made OS independent.
+Executor.WorkingDirectory=C:\\Windows\\Temp
+Executor.ApprovedExecutables=C:\\Windows\\System32\\cmd.exe,C:\\Windows\\System32\\runas.exe
+
+
+#===========================================================================
+# ESAPI Logging
+# Set the application name if these logs are combined with other applications
+Logger.ApplicationName=ExampleApplication
+# If you use an HTML log viewer that does not properly HTML escape log data, you can set LogEncodingRequired to true
+Logger.LogEncodingRequired=false
+# Determines whether ESAPI should log the application name. This might be clutter in some single-server/single-app environments.
+Logger.LogApplicationName=true
+# Determines whether ESAPI should log the server IP and port. This might be clutter in some single-server environments.
+Logger.LogServerIP=true
+# LogFileName, the name of the logging file. Provide a full directory path (e.g., C:\\ESAPI\\ESAPI_logging_file) if you
+# want to place it in a specific directory.
+Logger.LogFileName=ESAPI_logging_file
+# MaxLogFileSize, the max size (in bytes) of a single log file before it cuts over to a new one (default is 10,000,000)
+Logger.MaxLogFileSize=10000000
+
+
+#===========================================================================
+# ESAPI Intrusion Detection
+#
+# Each event has a base to which .count, .interval, and .action are added
+# The IntrusionException will fire if we receive "count" events within "interval" seconds
+# The IntrusionDetector is configurable to take the following actions: log, logout, and disable
+#  (multiple actions separated by commas are allowed e.g. event.test.actions=log,disable
+#
+# Custom Events
+# Names must start with "event." as the base
+# Use IntrusionDetector.addEvent( "test" ) in your code to trigger "event.test" here
+# You can also disable intrusion detection completely by changing
+# the following parameter to true
+#
+IntrusionDetector.Disable=false
+#
+IntrusionDetector.event.test.count=2
+IntrusionDetector.event.test.interval=10
+IntrusionDetector.event.test.actions=disable,log
+
+# Exception Events
+# All EnterpriseSecurityExceptions are registered automatically
+# Call IntrusionDetector.getInstance().addException(e) for Exceptions that do not extend EnterpriseSecurityException
+# Use the fully qualified classname of the exception as the base
+
+# any intrusion is an attack
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.count=1
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.interval=1
+IntrusionDetector.org.owasp.esapi.errors.IntrusionException.actions=log,disable,logout
+
+# for test purposes
+# CHECKME: Shouldn't there be something in the property name itself that designates
+#           that these are for testing???
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.count=10
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.interval=5
+IntrusionDetector.org.owasp.esapi.errors.IntegrityException.actions=log,disable,logout
+
+# rapid validation errors indicate scans or attacks in progress
+# org.owasp.esapi.errors.ValidationException.count=10
+# org.owasp.esapi.errors.ValidationException.interval=10
+# org.owasp.esapi.errors.ValidationException.actions=log,logout
+
+# sessions jumping between hosts indicates session hijacking
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.count=2
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.interval=10
+IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.actions=log,logout
+
+
+#===========================================================================
+# ESAPI Validation
+#
+# The ESAPI Validator works on regular expressions with defined names. You can define names
+# either here, or you may define application specific patterns in a separate file defined below.
+# This allows enterprises to specify both organizational standards as well as application specific
+# validation rules.
+#
+Validator.ConfigurationFile=validation.properties
+
+# Validators used by ESAPI
+Validator.AccountName=^[a-zA-Z0-9]{3,20}$
+Validator.SystemCommand=^[a-zA-Z\\-\\/]{1,64}$
+Validator.RoleName=^[a-z]{1,20}$
+
+#the word TEST below should be changed to your application
+#name - only relative URL's are supported
+Validator.Redirect=^\\/test.*$
+
+# Global HTTP Validation Rules
+# Values with Base64 encoded data (e.g. encrypted state) will need at least [a-zA-Z0-9\/+=]
+Validator.HTTPScheme=^(http|https)$
+Validator.HTTPServerName=^[a-zA-Z0-9_.\\-]*$
+Validator.HTTPParameterName=^[a-zA-Z0-9_]{1,32}$
+Validator.HTTPParameterValue=^[a-zA-Z0-9.\\-\\/+=_ ]*$
+Validator.HTTPCookieName=^[a-zA-Z0-9\\-_]{1,32}$
+Validator.HTTPCookieValue=^[a-zA-Z0-9\\-\\/+=_ ]*$
+Validator.HTTPHeaderName=^[a-zA-Z0-9\\-_]{1,32}$
+Validator.HTTPHeaderValue=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
+Validator.HTTPContextPath=^[a-zA-Z0-9.\\-\\/_]*$
+Validator.HTTPServletPath=^[a-zA-Z0-9.\\-\\/_]*$
+Validator.HTTPPath=^[a-zA-Z0-9.\\-_]*$
+Validator.HTTPQueryString=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ %]*$
+Validator.HTTPURI=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
+Validator.HTTPURL=^.*$
+Validator.HTTPJSESSIONID=^[A-Z0-9]{10,30}$
+
+# Validation of file related input
+Validator.FileName=^[a-zA-Z0-9!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$
+Validator.DirectoryName=^[a-zA-Z0-9:/\\\\!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$

http://git-wip-us.apache.org/repos/asf/hbase/blob/efc7a0d3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 87865d2..af49452 100644
--- a/pom.xml
+++ b/pom.xml
@@ -828,6 +828,7 @@
               <exclude>**/patchprocess/**</exclude>
               <exclude>src/main/site/resources/repo/**</exclude>
               <exclude>**/dependency-reduced-pom.xml</exclude>
+              <exclude>**/ESAPI.properties</exclude>
               <exclude>**/rat.txt</exclude>
             </excludes>
           </configuration>


[20/32] hbase git commit: HBASE-15157 Add *PerformanceTest for Append, CheckAnd*

Posted by sy...@apache.org.
HBASE-15157 Add *PerformanceTest for Append, CheckAnd*


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

Branch: refs/heads/hbase-12439
Commit: 81d81c9839118113e8076338315bf6c500065c09
Parents: 7239056
Author: stack <st...@apache.org>
Authored: Fri Feb 5 11:18:42 2016 -0800
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 20:33:55 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/IncrementPerformanceTest.java  | 128 ----------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 148 ++++++++++++++++++-
 2 files changed, 142 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/81d81c98/hbase-server/src/test/java/org/apache/hadoop/hbase/IncrementPerformanceTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/IncrementPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/IncrementPerformanceTest.java
deleted file mode 100644
index aed3d0a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/IncrementPerformanceTest.java
+++ /dev/null
@@ -1,128 +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;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Snapshot;
-import com.codahale.metrics.Timer;
-
-
-/**
- * Simple Increments Performance Test. Run this from main. It is to go against a cluster.
- * Presumption is the table exists already. Defaults are a zk ensemble of localhost:2181,
- * a tableName of 'tableName', a column famly name of 'columnFamilyName', with 80 threads by
- * default and 10000 increments per thread. To change any of these configs, pass -DNAME=VALUE as
- * in -DtableName="newTableName". It prints out configuration it is running with at the start and
- * on the end it prints out percentiles.
- */
-public class IncrementPerformanceTest implements Tool {
-  private static final Log LOG = LogFactory.getLog(IncrementPerformanceTest.class);
-  private static final byte [] QUALIFIER = new byte [] {'q'};
-  private Configuration conf;
-  private final MetricRegistry metrics = new MetricRegistry();
-  private static final String TABLENAME = "tableName";
-  private static final String COLUMN_FAMILY = "columnFamilyName";
-  private static final String THREAD_COUNT = "threadCount";
-  private static final int DEFAULT_THREAD_COUNT = 80;
-  private static final String INCREMENT_COUNT = "incrementCount";
-  private static final int DEFAULT_INCREMENT_COUNT = 10000;
-
-  IncrementPerformanceTest() {}
-
-  public int run(final String [] args) throws Exception {
-    Configuration conf = getConf();
-    final TableName tableName = TableName.valueOf(conf.get(TABLENAME), TABLENAME);
-    final byte [] columnFamilyName = Bytes.toBytes(conf.get(COLUMN_FAMILY, COLUMN_FAMILY));
-    int threadCount = conf.getInt(THREAD_COUNT, DEFAULT_THREAD_COUNT);
-    final int incrementCount = conf.getInt(INCREMENT_COUNT, DEFAULT_INCREMENT_COUNT);
-    LOG.info("Running test with " + HConstants.ZOOKEEPER_QUORUM + "=" +
-      getConf().get(HConstants.ZOOKEEPER_QUORUM) + ", tableName=" + tableName +
-      ", columnFamilyName=" + columnFamilyName + ", threadCount=" + threadCount +
-      ", incrementCount=" + incrementCount);
-
-    ExecutorService service = Executors.newFixedThreadPool(threadCount);
-    Set<Future<?>> futures = new HashSet<Future<?>>();
-    final AtomicInteger integer = new AtomicInteger(0); // needed a simple "final" counter
-    while (integer.incrementAndGet() <= threadCount) {
-      futures.add(service.submit(new Runnable() {
-        @Override
-        public void run() {
-          try {
-            try (Connection connection = ConnectionFactory.createConnection(getConf())) {
-              try (Table table = connection.getTable(tableName)) {
-                Timer timer = metrics.timer("increments");
-                for (int i = 0; i < incrementCount; i++) {
-                  byte[] row = Bytes.toBytes(i);
-                  Timer.Context context = timer.time();
-                  try {
-                    table.incrementColumnValue(row, columnFamilyName, QUALIFIER, 1l);
-                  } catch (IOException e) {
-                    // swallow..it's a test.
-                  } finally {
-                    context.stop();
-                  }
-                }
-              }
-            }
-          } catch (IOException ioe) {
-            throw new RuntimeException(ioe);
-          }
-        }
-      }));
-    }
-
-    for(Future<?> future : futures) future.get();
-    service.shutdown();
-    Snapshot s = metrics.timer("increments").getSnapshot();
-    LOG.info(String.format("75th=%s, 95th=%s, 99th=%s", s.get75thPercentile(),
-        s.get95thPercentile(), s.get99thPercentile()));
-    return 0;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return this.conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(HBaseConfiguration.create(), new IncrementPerformanceTest(), args));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/81d81c98/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 821b995..651bc86 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -49,19 +49,24 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -165,7 +170,17 @@ public class PerformanceEvaluation extends Configured implements Tool {
       "Run scan test (read every row)");
     addCommandDescriptor(FilteredScanTest.class, "filterScan",
       "Run scan test using a filter to find a specific row based on it's value " +
-        "(make sure to use --rows=20)");
+      "(make sure to use --rows=20)");
+    addCommandDescriptor(IncrementTest.class, "increment",
+      "Increment on each row; clients overlap on keyspace so some concurrent operations");
+    addCommandDescriptor(AppendTest.class, "append",
+      "Append on each row; clients overlap on keyspace so some concurrent operations");
+    addCommandDescriptor(CheckAndMutateTest.class, "checkAndMutate",
+      "CheckAndMutate on each row; clients overlap on keyspace so some concurrent operations");
+    addCommandDescriptor(CheckAndPutTest.class, "checkAndPut",
+      "CheckAndPut on each row; clients overlap on keyspace so some concurrent operations");
+    addCommandDescriptor(CheckAndDeleteTest.class, "checkAndDelete",
+      "CheckAndDelete on each row; clients overlap on keyspace so some concurrent operations");
   }
 
   /**
@@ -1089,15 +1104,24 @@ public class PerformanceEvaluation extends Configured implements Tool {
       return (System.nanoTime() - startTime) / 1000000;
     }
 
+    int getStartRow() {
+      return opts.startRow;
+    }
+
+    int getLastRow() {
+      return getStartRow() + opts.perClientRunRows;
+    }
+
     /**
      * Provides an extension point for tests that don't want a per row invocation.
      */
     void testTimed() throws IOException, InterruptedException {
-      int lastRow = opts.startRow + opts.perClientRunRows;
+      int startRow = getStartRow();
+      int lastRow = getLastRow();
       // Report on completion of 1/10th of total.
       for (int ii = 0; ii < opts.cycles; ii++) {
         if (opts.cycles > 1) LOG.info("Cycle=" + ii + " of " + opts.cycles);
-        for (int i = opts.startRow; i < lastRow; i++) {
+        for (int i = startRow; i < lastRow; i++) {
           if (i % everyN != 0) continue;
           long startTime = System.nanoTime();
           TraceScope scope = Trace.startSpan("test row", traceSampler);
@@ -1106,15 +1130,16 @@ public class PerformanceEvaluation extends Configured implements Tool {
           } finally {
             scope.close();
           }
-          if ( (i - opts.startRow) > opts.measureAfter) {
+          if ( (i - startRow) > opts.measureAfter) {
             latency.update((System.nanoTime() - startTime) / 1000);
             if (status != null && i > 0 && (i % getReportingPeriod()) == 0) {
-              status.setStatus(generateStatus(opts.startRow, i, lastRow));
+              status.setStatus(generateStatus(startRow, i, lastRow));
             }
           }
         }
       }
     }
+
     /**
      * report percentiles of latency
      * @throws IOException
@@ -1456,7 +1481,116 @@ public class PerformanceEvaluation extends Configured implements Tool {
       Result r = testScanner.next();
       updateValueSize(r);
     }
+  }
+
+  /**
+   * Base class for operations that are CAS-like; that read a value and then set it based off what
+   * they read. In this category is increment, append, checkAndPut, etc.
+   *
+   * <p>These operations also want some concurrency going on. Usually when these tests run, they
+   * operate in their own part of the key range. In CASTest, we will have them all overlap on the
+   * same key space. We do this with our getStartRow and getLastRow overrides.
+   */
+  static abstract class CASTableTest extends TableTest {
+    private final byte [] qualifier;
+    CASTableTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+      qualifier = Bytes.toBytes(this.getClass().getSimpleName());
+    }
+
+    byte [] getQualifier() {
+      return this.qualifier;
+    }
+
+    @Override
+    int getStartRow() {
+      return 0;
+    }
+
+    @Override
+    int getLastRow() {
+      return opts.perClientRunRows;
+    }
+  }
+
+  static class IncrementTest extends CASTableTest {
+    IncrementTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      Increment increment = new Increment(format(i));
+      increment.addColumn(FAMILY_NAME, getQualifier(), 1l);
+      updateValueSize(this.table.increment(increment));
+    }
+  }
+
+  static class AppendTest extends CASTableTest {
+    AppendTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      byte [] bytes = format(i);
+      Append append = new Append(bytes);
+      append.add(FAMILY_NAME, getQualifier(), bytes);
+      updateValueSize(this.table.append(append));
+    }
+  }
+
+  static class CheckAndMutateTest extends CASTableTest {
+    CheckAndMutateTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      byte [] bytes = format(i);
+      // Put a known value so when we go to check it, it is there.
+      Put put = new Put(bytes);
+      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      this.table.put(put);
+      RowMutations mutations = new RowMutations(bytes);
+      mutations.add(put);
+      this.table.checkAndMutate(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes,
+          mutations);
+    }
+  }
 
+  static class CheckAndPutTest extends CASTableTest {
+    CheckAndPutTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      byte [] bytes = format(i);
+      // Put a known value so when we go to check it, it is there.
+      Put put = new Put(bytes);
+      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      this.table.put(put);
+      this.table.checkAndPut(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes, put);
+    }
+  }
+
+  static class CheckAndDeleteTest extends CASTableTest {
+    CheckAndDeleteTest(Connection con, TestOptions options, Status status) {
+      super(con, options, status);
+    }
+
+    @Override
+    void testRow(final int i) throws IOException {
+      byte [] bytes = format(i);
+      // Put a known value so when we go to check it, it is there.
+      Put put = new Put(bytes);
+      put.addColumn(FAMILY_NAME, getQualifier(), bytes);
+      this.table.put(put);
+      Delete delete = new Delete(put.getRow());
+      delete.addColumn(FAMILY_NAME, getQualifier());
+      this.table.checkAndDelete(bytes, FAMILY_NAME, getQualifier(), CompareOp.EQUAL, bytes, delete);
+    }
   }
 
   static class SequentialReadTest extends TableTest {
@@ -1760,8 +1894,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
       "clients (and HRegionServers)");
     System.err.println("                 running: 1 <= value <= 500");
     System.err.println("Examples:");
-    System.err.println(" To run a single evaluation client:");
+    System.err.println(" To run a single client doing the default 1M sequentialWrites:");
     System.err.println(" $ bin/hbase " + className + " sequentialWrite 1");
+    System.err.println(" To run 10 clients doing increments over ten rows:");
+    System.err.println(" $ bin/hbase " + className + " --rows=10 --nomapred increment 10");
   }
 
   /**


[10/32] hbase git commit: HBASE-15210 Undo aggressive load balancer logging at tens of lines per millisecond; ADDENDUM changing LOG.debug to LOG.trace -- noticed by matteo

Posted by sy...@apache.org.
 HBASE-15210 Undo aggressive load balancer logging at tens of lines per millisecond; ADDENDUM changing LOG.debug to LOG.trace -- noticed by matteo


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

Branch: refs/heads/hbase-12439
Commit: f4d9597e1eef3baa9266581cf762ae3b249a1dff
Parents: 18eff3c
Author: stack <st...@apache.org>
Authored: Thu Feb 4 16:23:32 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 4 16:26:04 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f4d9597e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 8680c89..44e1f79 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -844,7 +844,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           }
         }
         if (LOG.isTraceEnabled()) {
-          LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
+          LOG.trace(" Lowest locality region index is " + lowestLocalityRegionIndex
             + " and its region server contains " + regionsPerServer[serverIndex].length
             + " regions");
         }


[05/32] hbase git commit: Fixup on the hbasecon banner image.. add date and location

Posted by sy...@apache.org.
Fixup on the hbasecon banner image.. add date and location


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

Branch: refs/heads/hbase-12439
Commit: d5d26f0804f0f6ed7feb4d23fbc1fddd189b2f71
Parents: 7b33a74
Author: stack <st...@apache.org>
Authored: Thu Feb 4 11:44:16 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 4 11:44:16 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java    | 6 +++---
 src/main/site/site.xml                                         | 2 +-
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d5d26f08/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 0bc75eb..2bf4119 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -317,7 +317,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
           if (perms != Perms.ALL) {
             if (LOG.isDebugEnabled()) {
               LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
-                id, perms, Perms.ALL));
+                id.toString(), perms, Perms.ALL));
             }
             return false;
           }
@@ -336,7 +336,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     }
     return true;
   }
-  
+
   /*
    * Validate whether ACL set for all superusers.
    */
@@ -366,7 +366,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     }
     return true;
   }
-  
+
   /*
    * Validate whether ACL ID is superuser.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/d5d26f08/src/main/site/site.xml
----------------------------------------------------------------------
diff --git a/src/main/site/site.xml b/src/main/site/site.xml
index b847293..02b28ca 100644
--- a/src/main/site/site.xml
+++ b/src/main/site/site.xml
@@ -48,7 +48,7 @@
     <height>0</height>
     <width>0</width>
     -->
-    <src>images/hbasecon2016-stack-logo.jpg</src>
+    <src>images/hbasecon2016-stacked.png</src>
     <href>http://hbasecon.com/</href>
   </bannerLeft>
   <bannerRight>


[27/32] hbase git commit: HBASE-15163 Add sampling code and metrics for get/scan/multi/mutate count separately (Yu Li)

Posted by sy...@apache.org.
HBASE-15163 Add sampling code and metrics for get/scan/multi/mutate count separately (Yu Li)


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

Branch: refs/heads/hbase-12439
Commit: eacf7bcf97f09c9a6e68baf9a4a9ceb1d83c9fb0
Parents: d82ae42
Author: stack <st...@apache.org>
Authored: Sat Feb 6 06:30:56 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sat Feb 6 06:30:56 2016 -0800

----------------------------------------------------------------------
 .../regionserver/MetricsRegionServerSource.java | 12 ++++++++++++
 .../MetricsRegionServerWrapper.java             | 20 ++++++++++++++++++++
 .../MetricsRegionServerSourceImpl.java          |  8 ++++++++
 .../hbase/regionserver/HRegionServer.java       |  4 ++++
 .../MetricsRegionServerWrapperImpl.java         | 20 ++++++++++++++++++++
 .../hbase/regionserver/RSRpcServices.java       | 17 +++++++++++++++++
 .../MetricsRegionServerWrapperStub.java         | 20 ++++++++++++++++++++
 7 files changed, 101 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
index e4df1c0..0f2f90b 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
@@ -330,4 +330,16 @@ public interface MetricsRegionServerSource extends BaseSource {
   String SPLIT_SUCCESS_KEY = "splitSuccessCount";
   String SPLIT_SUCCESS_DESC = "Number of successfully executed splits";
   String FLUSH_KEY = "flushTime";
+
+  String RPC_GET_REQUEST_COUNT = "rpcGetRequestCount";
+  String RPC_GET_REQUEST_COUNT_DESC = "Number of rpc get requests this region server has answered.";
+  String RPC_SCAN_REQUEST_COUNT = "rpcScanRequestCount";
+  String RPC_SCAN_REQUEST_COUNT_DESC =
+      "Number of rpc scan requests this region server has answered.";
+  String RPC_MULTI_REQUEST_COUNT = "rpcMultiRequestCount";
+  String RPC_MULTI_REQUEST_COUNT_DESC =
+      "Number of rpc multi requests this region server has answered.";
+  String RPC_MUTATE_REQUEST_COUNT = "rpcMutateRequestCount";
+  String RPC_MUTATE_REQUEST_COUNT_DESC =
+      "Number of rpc mutation requests this region server has answered.";
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
index 07c3773..ee2b5a1 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java
@@ -372,4 +372,24 @@ public interface MetricsRegionServerWrapper {
    * @return Count of requests blocked because the memstore size is larger than blockingMemStoreSize
    */
   long getBlockedRequestsCount();
+
+  /**
+   * Get the number of rpc get requests to this region server.
+   */
+  long getRpcGetRequestsCount();
+
+  /**
+   * Get the number of rpc scan requests to this region server.
+   */
+  long getRpcScanRequestsCount();
+
+  /**
+   * Get the number of rpc multi requests to this region server.
+   */
+  long getRpcMultiRequestsCount();
+
+  /**
+   * Get the number of rpc mutate requests to this region server.
+   */
+  long getRpcMutateRequestsCount();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index 42476a7..9134f46 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -205,6 +205,14 @@ public class MetricsRegionServerSourceImpl
               rsWrap.getFilteredReadRequestsCount())
           .addCounter(Interns.info(WRITE_REQUEST_COUNT, WRITE_REQUEST_COUNT_DESC),
               rsWrap.getWriteRequestsCount())
+          .addCounter(Interns.info(RPC_GET_REQUEST_COUNT, RPC_GET_REQUEST_COUNT_DESC),
+            rsWrap.getRpcGetRequestsCount())
+          .addCounter(Interns.info(RPC_SCAN_REQUEST_COUNT, RPC_SCAN_REQUEST_COUNT_DESC),
+            rsWrap.getRpcScanRequestsCount())
+          .addCounter(Interns.info(RPC_MULTI_REQUEST_COUNT, RPC_MULTI_REQUEST_COUNT_DESC),
+            rsWrap.getRpcMultiRequestsCount())
+          .addCounter(Interns.info(RPC_MUTATE_REQUEST_COUNT, RPC_MUTATE_REQUEST_COUNT_DESC),
+            rsWrap.getRpcMutateRequestsCount())
           .addCounter(Interns.info(CHECK_MUTATE_FAILED_COUNT, CHECK_MUTATE_FAILED_COUNT_DESC),
               rsWrap.getCheckAndMutateChecksFailed())
           .addCounter(Interns.info(CHECK_MUTATE_PASSED_COUNT, CHECK_MUTATE_PASSED_COUNT_DESC),

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 9cb100f..4ab2693 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -2331,6 +2331,10 @@ public class HRegionServer extends HasThread implements
     RegionServerStartupResponse result = null;
     try {
       rpcServices.requestCount.set(0);
+      rpcServices.rpcGetRequestCount.set(0);
+      rpcServices.rpcScanRequestCount.set(0);
+      rpcServices.rpcMultiRequestCount.set(0);
+      rpcServices.rpcMutateRequestCount.set(0);
       LOG.info("reportForDuty to master=" + masterServerName + " with port="
         + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
       long now = EnvironmentEdgeManager.currentTime();

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index 4f9ba5b..777c960 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -419,6 +419,26 @@ class MetricsRegionServerWrapperImpl
   }
 
   @Override
+  public long getRpcGetRequestsCount() {
+    return regionServer.rpcServices.rpcGetRequestCount.get();
+  }
+
+  @Override
+  public long getRpcScanRequestsCount() {
+    return regionServer.rpcServices.rpcScanRequestCount.get();
+  }
+
+  @Override
+  public long getRpcMultiRequestsCount() {
+    return regionServer.rpcServices.rpcMultiRequestCount.get();
+  }
+
+  @Override
+  public long getRpcMutateRequestsCount() {
+    return regionServer.rpcServices.rpcMutateRequestCount.get();
+  }
+
+  @Override
   public long getCheckAndMutateChecksFailed() {
     return checkAndMutateChecksFailed;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/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 e346c34..7bcde52 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
@@ -220,6 +220,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
   // Request counter. (Includes requests that are not serviced by regions.)
   final Counter requestCount = new Counter();
+
+  // Request counter for rpc get
+  final Counter rpcGetRequestCount = new Counter();
+
+  // Request counter for rpc scan
+  final Counter rpcScanRequestCount = new Counter();
+
+  // Request counter for rpc multi
+  final Counter rpcMultiRequestCount = new Counter();
+
+  // Request counter for rpc mutate
+  final Counter rpcMutateRequestCount = new Counter();
+
   // Server to handle client requests.
   final RpcServerInterface rpcServer;
   final InetSocketAddress isa;
@@ -2091,6 +2104,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
+      rpcGetRequestCount.increment();
       Region region = getRegion(request.getRegion());
 
       GetResponse.Builder builder = GetResponse.newBuilder();
@@ -2231,6 +2245,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     Boolean processed = null;
     RegionScannersCloseCallBack closeCallBack = null;
     RpcCallContext context = RpcServer.getCurrentCall();
+    this.rpcMultiRequestCount.increment();
     for (RegionAction regionAction : request.getRegionActionList()) {
       this.requestCount.add(regionAction.getActionCount());
       OperationQuota quota;
@@ -2352,6 +2367,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     try {
       checkOpen();
       requestCount.increment();
+      rpcMutateRequestCount.increment();
       Region region = getRegion(request.getRegion());
       MutateResponse.Builder builder = MutateResponse.newBuilder();
       MutationProto mutation = request.getMutation();
@@ -2496,6 +2512,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         throw e;
       }
       requestCount.increment();
+      rpcScanRequestCount.increment();
 
       int ttl = 0;
       Region region = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eacf7bcf/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
index 4b00632..95a7cf5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java
@@ -96,6 +96,26 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
   }
 
   @Override
+  public long getRpcGetRequestsCount() {
+    return 521;
+  }
+
+  @Override
+  public long getRpcScanRequestsCount() {
+    return 101;
+  }
+
+  @Override
+  public long getRpcMultiRequestsCount() {
+    return 486;
+  }
+
+  @Override
+  public long getRpcMutateRequestsCount() {
+    return 606;
+  }
+
+  @Override
   public long getCheckAndMutateChecksFailed() {
     return 401;
   }


[25/32] hbase git commit: HBASE-15204 Try to estimate the cell count for adding into WALEdit (Revert for making it more cleaner)

Posted by sy...@apache.org.
HBASE-15204 Try to estimate the cell count for adding into WALEdit (Revert
for making it more cleaner)


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

Branch: refs/heads/hbase-12439
Commit: 4e44f4f5050bf2720762f754d3756763026c0dbd
Parents: 59b03c7
Author: ramkrishna <ra...@gmail.com>
Authored: Sat Feb 6 13:05:13 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Sat Feb 6 13:05:13 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 25 +++++++-------------
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |  8 +------
 2 files changed, 10 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4e44f4f5/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 86f4a1b..f03c205 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
@@ -2906,26 +2906,25 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    *         OperationStatusCode and the exceptionMessage if any.
    * @throws IOException
    */
-  OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp)
-      throws IOException {
+  OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
     boolean initialized = false;
     Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
     startRegionOperation(op);
-    int cellCountFromCP = 0;
     try {
       while (!batchOp.isDone()) {
         if (!batchOp.isInReplay()) {
           checkReadOnly();
         }
         checkResources();
+
         if (!initialized) {
           this.writeRequestsCount.add(batchOp.operations.length);
           if (!batchOp.isInReplay()) {
-            cellCountFromCP = doPreMutationHook(batchOp);
+            doPreMutationHook(batchOp);
           }
           initialized = true;
         }
-        long addedSize = doMiniBatchMutation(batchOp, cellCountFromCP);
+        long addedSize = doMiniBatchMutation(batchOp);
         long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
         if (isFlushSize(newSize)) {
           requestFlush();
@@ -2938,11 +2937,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
 
-  private int doPreMutationHook(BatchOperationInProgress<?> batchOp)
+  private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
       throws IOException {
     /* Run coprocessor pre hook outside of locks to avoid deadlock */
     WALEdit walEdit = new WALEdit();
-    int cellCount = 0;
     if (coprocessorHost != null) {
       for (int i = 0 ; i < batchOp.operations.length; i++) {
         Mutation m = batchOp.getMutation(i);
@@ -2972,17 +2970,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
         if (!walEdit.isEmpty()) {
           batchOp.walEditsFromCoprocessors[i] = walEdit;
-          cellCount += walEdit.size();
           walEdit = new WALEdit();
         }
       }
     }
-    return cellCount;
   }
 
   @SuppressWarnings("unchecked")
-  private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp, int cellCount)
-      throws IOException {
+  private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
     boolean isInReplay = batchOp.isInReplay();
     // variable to note if all Put items are for the same CF -- metrics related
     boolean putsCfSetConsistent = true;
@@ -2994,7 +2989,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     Set<byte[]> deletesCfSet = null;
 
     long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
-    WALEdit walEdit = null;
+    WALEdit walEdit = new WALEdit(isInReplay);
     MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
     long txid = 0;
     boolean doRollBackMemstore = false;
@@ -3025,6 +3020,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
         // store the family map reference to allow for mutations
         familyMaps[lastIndexExclusive] = familyMap;
+
         // skip anything that "ran" already
         if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
             != OperationStatusCode.NOT_RUN) {
@@ -3131,11 +3127,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           noOfDeletes++;
         }
         rewriteCellTags(familyMaps[i], mutation);
-        for (List<Cell> cells : familyMaps[i].values()) {
-          cellCount += cells.size();
-        }
       }
-      walEdit = new WALEdit(cellCount);
+
       lock(this.updatesLock.readLock(), numReadyToWrite);
       locked = true;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4e44f4f5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index 1a87447..cea2ee7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -99,7 +99,7 @@ public class WALEdit implements Writable, HeapSize {
   private final int VERSION_2 = -1;
   private final boolean isReplay;
 
-  private ArrayList<Cell> cells = null;
+  private ArrayList<Cell> cells = new ArrayList<Cell>(1);
 
   public static final WALEdit EMPTY_WALEDIT = new WALEdit();
 
@@ -118,12 +118,6 @@ public class WALEdit implements Writable, HeapSize {
 
   public WALEdit(boolean isReplay) {
     this.isReplay = isReplay;
-    cells = new ArrayList<Cell>(1);
-  }
-
-  public WALEdit(int cellCount) {
-    this.isReplay = false;
-    cells = new ArrayList<Cell>(cellCount);
   }
 
   /**


[32/32] hbase git commit: HBASE-15201 Add hbase-spark to hbase assembly

Posted by sy...@apache.org.
HBASE-15201 Add hbase-spark to hbase assembly


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

Branch: refs/heads/hbase-12439
Commit: 3aff98c75b5e23a5010be17eecef3140d2bf70bb
Parents: ec92a8a
Author: Jerry He <je...@apache.org>
Authored: Mon Feb 8 14:13:46 2016 -0800
Committer: Jerry He <je...@apache.org>
Committed: Mon Feb 8 14:13:46 2016 -0800

----------------------------------------------------------------------
 hbase-assembly/pom.xml                                 | 5 +++++
 hbase-assembly/src/main/assembly/hadoop-two-compat.xml | 1 +
 hbase-spark/pom.xml                                    | 1 +
 3 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3aff98c7/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 4851391..87e82ad 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -201,5 +201,10 @@
        <version>${project.version}</version>
        <optional>true</optional>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-spark</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aff98c7/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 9ef624c..2033e9c 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -45,6 +45,7 @@
         <include>org.apache.hbase:hbase-rest</include>
         <include>org.apache.hbase:hbase-server</include>
         <include>org.apache.hbase:hbase-shell</include>
+        <include>org.apache.hbase:hbase-spark</include>
         <include>org.apache.hbase:hbase-thrift</include>
         <include>org.apache.hbase:hbase-external-blockcache</include>
       </includes>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aff98c7/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 8f71a89..251ea59 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -88,6 +88,7 @@
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-streaming_${scala.binary.version}</artifactId>
             <version>${spark.version}</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.spark</groupId>


[14/32] hbase git commit: HBASE-15204 Try to estimate the cell count for adding into WALEdit (Ram)

Posted by sy...@apache.org.
HBASE-15204 Try to estimate the cell count for adding into WALEdit (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 6f6a8ed71fe98b83e8a8db974fc15b0d8597b174
Parents: bb71446
Author: ramkrishna <ra...@gmail.com>
Authored: Fri Feb 5 14:23:36 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Fri Feb 5 14:24:38 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 25 +++++++++++++-------
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |  8 ++++++-
 2 files changed, 23 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6f6a8ed7/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 f03c205..86f4a1b 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
@@ -2906,25 +2906,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    *         OperationStatusCode and the exceptionMessage if any.
    * @throws IOException
    */
-  OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
+  OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp)
+      throws IOException {
     boolean initialized = false;
     Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
     startRegionOperation(op);
+    int cellCountFromCP = 0;
     try {
       while (!batchOp.isDone()) {
         if (!batchOp.isInReplay()) {
           checkReadOnly();
         }
         checkResources();
-
         if (!initialized) {
           this.writeRequestsCount.add(batchOp.operations.length);
           if (!batchOp.isInReplay()) {
-            doPreMutationHook(batchOp);
+            cellCountFromCP = doPreMutationHook(batchOp);
           }
           initialized = true;
         }
-        long addedSize = doMiniBatchMutation(batchOp);
+        long addedSize = doMiniBatchMutation(batchOp, cellCountFromCP);
         long newSize = this.addAndGetGlobalMemstoreSize(addedSize);
         if (isFlushSize(newSize)) {
           requestFlush();
@@ -2937,10 +2938,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
 
-  private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
+  private int doPreMutationHook(BatchOperationInProgress<?> batchOp)
       throws IOException {
     /* Run coprocessor pre hook outside of locks to avoid deadlock */
     WALEdit walEdit = new WALEdit();
+    int cellCount = 0;
     if (coprocessorHost != null) {
       for (int i = 0 ; i < batchOp.operations.length; i++) {
         Mutation m = batchOp.getMutation(i);
@@ -2970,14 +2972,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
         if (!walEdit.isEmpty()) {
           batchOp.walEditsFromCoprocessors[i] = walEdit;
+          cellCount += walEdit.size();
           walEdit = new WALEdit();
         }
       }
     }
+    return cellCount;
   }
 
   @SuppressWarnings("unchecked")
-  private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
+  private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp, int cellCount)
+      throws IOException {
     boolean isInReplay = batchOp.isInReplay();
     // variable to note if all Put items are for the same CF -- metrics related
     boolean putsCfSetConsistent = true;
@@ -2989,7 +2994,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     Set<byte[]> deletesCfSet = null;
 
     long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
-    WALEdit walEdit = new WALEdit(isInReplay);
+    WALEdit walEdit = null;
     MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
     long txid = 0;
     boolean doRollBackMemstore = false;
@@ -3020,7 +3025,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
         // store the family map reference to allow for mutations
         familyMaps[lastIndexExclusive] = familyMap;
-
         // skip anything that "ran" already
         if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
             != OperationStatusCode.NOT_RUN) {
@@ -3127,8 +3131,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           noOfDeletes++;
         }
         rewriteCellTags(familyMaps[i], mutation);
+        for (List<Cell> cells : familyMaps[i].values()) {
+          cellCount += cells.size();
+        }
       }
-
+      walEdit = new WALEdit(cellCount);
       lock(this.updatesLock.readLock(), numReadyToWrite);
       locked = true;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6f6a8ed7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index cea2ee7..1a87447 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -99,7 +99,7 @@ public class WALEdit implements Writable, HeapSize {
   private final int VERSION_2 = -1;
   private final boolean isReplay;
 
-  private ArrayList<Cell> cells = new ArrayList<Cell>(1);
+  private ArrayList<Cell> cells = null;
 
   public static final WALEdit EMPTY_WALEDIT = new WALEdit();
 
@@ -118,6 +118,12 @@ public class WALEdit implements Writable, HeapSize {
 
   public WALEdit(boolean isReplay) {
     this.isReplay = isReplay;
+    cells = new ArrayList<Cell>(1);
+  }
+
+  public WALEdit(int cellCount) {
+    this.isReplay = false;
+    cells = new ArrayList<Cell>(cellCount);
   }
 
   /**


[09/32] hbase git commit: HBASE-15210 Undo aggressive load balancer logging at tens of lines per millisecond

Posted by sy...@apache.org.
HBASE-15210 Undo aggressive load balancer logging at tens of lines per millisecond


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

Branch: refs/heads/hbase-12439
Commit: 18eff3c1c337003b2a419490e621f931d16936fb
Parents: 337f483
Author: stack <st...@apache.org>
Authored: Thu Feb 4 16:09:26 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 4 16:09:26 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/balancer/BaseLoadBalancer.java       | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/18eff3c1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 20f4169..8680c89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -816,9 +816,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           i++;
           lowestLocalityServerIndex = serverIndicesSortedByLocality[i];
         }
-        LOG.debug("Lowest locality region server with non zero regions is "
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Lowest locality region server with non zero regions is "
             + servers[lowestLocalityServerIndex].getHostname() + " with locality "
             + localityPerServer[lowestLocalityServerIndex]);
+        }
         return lowestLocalityServerIndex;
       }
     }
@@ -841,9 +843,11 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
             lowestLocalityRegionIndex = j;
           }
         }
-        LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
+        if (LOG.isTraceEnabled()) {
+          LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
             + " and its region server contains " + regionsPerServer[serverIndex].length
             + " regions");
+        }
         return regionsPerServer[serverIndex][lowestLocalityRegionIndex];
       } else {
         return -1;


[08/32] hbase git commit: Revert "HBASE-15120 Undo aggressive load balancer logging at tens of lines per millisecond"

Posted by sy...@apache.org.
Revert "HBASE-15120 Undo aggressive load balancer logging at tens of lines per millisecond"

This reverts commit c177cfed416e0cf72ae6131c5e98c99672bef3cc.


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

Branch: refs/heads/hbase-12439
Commit: 337f4830770d87e2fe629d742a3dcc64dba0c2cb
Parents: c177cfe
Author: stack <st...@apache.org>
Authored: Thu Feb 4 16:08:41 2016 -0800
Committer: stack <st...@apache.org>
Committed: Thu Feb 4 16:08:41 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/balancer/BaseLoadBalancer.java       | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/337f4830/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 8680c89..20f4169 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -816,11 +816,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           i++;
           lowestLocalityServerIndex = serverIndicesSortedByLocality[i];
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Lowest locality region server with non zero regions is "
+        LOG.debug("Lowest locality region server with non zero regions is "
             + servers[lowestLocalityServerIndex].getHostname() + " with locality "
             + localityPerServer[lowestLocalityServerIndex]);
-        }
         return lowestLocalityServerIndex;
       }
     }
@@ -843,11 +841,9 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
             lowestLocalityRegionIndex = j;
           }
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
+        LOG.debug(" Lowest locality region index is " + lowestLocalityRegionIndex
             + " and its region server contains " + regionsPerServer[serverIndex].length
             + " regions");
-        }
         return regionsPerServer[serverIndex][lowestLocalityRegionIndex];
       } else {
         return -1;


[11/32] hbase git commit: Amend HBASE-15200 ZooKeeper znode ACL checks should only compare the shortname

Posted by sy...@apache.org.
Amend HBASE-15200 ZooKeeper znode ACL checks should only compare the shortname

Fixes for newly introduced FindBugs warnings


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

Branch: refs/heads/hbase-12439
Commit: b80325fb1b6ad4735c8b2cb259b1430eddefd7b7
Parents: f4d9597
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Feb 4 16:17:41 2016 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Feb 4 17:33:32 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java   | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b80325fb/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
index 2bf4119..36a9bc5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
@@ -293,7 +293,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
       if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
         if (perms != Perms.READ) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
+            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
               id, perms, Perms.READ));
           }
           return false;
@@ -301,7 +301,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
       } else if (superUsers != null && isSuperUserId(superUsers, id)) {
         if (perms != Perms.ALL) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
+            LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
               id, perms, Perms.ALL));
           }
           return false;
@@ -316,8 +316,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
         if (name.equals(hbaseUser)) {
           if (perms != Perms.ALL) {
             if (LOG.isDebugEnabled()) {
-              LOG.debug(String.format("permissions for '%s' are not correct: have %0x, want %0x",
-                id.toString(), perms, Perms.ALL));
+              LOG.debug(String.format("permissions for '%s' are not correct: have 0x%x, want 0x%x",
+                id, perms, Perms.ALL));
             }
             return false;
           }
@@ -352,7 +352,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
             } else {
               if (LOG.isDebugEnabled()) {
                 LOG.debug(String.format(
-                  "superuser '%s' does not have correct permissions: have %0x, want %0x",
+                  "superuser '%s' does not have correct permissions: have 0x%x, want 0x%x",
                   acl.getId().getId(), acl.getPerms(), Perms.ALL));
               }
             }


[18/32] hbase git commit: HBASE-15017 Clean up stale GitHub PRs.

Posted by sy...@apache.org.
HBASE-15017 Clean up stale GitHub PRs.

* closes #10 (already applied via Jira)
* closes #23 (attempts merge branch-1.1 -> master)
* closes #2 (attempts merge 0.94 -> master)

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


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

Branch: refs/heads/hbase-12439
Commit: 4265bf275fede1fbc85bfcdfa39dad6e4b0afbe0
Parents: 779bdf1
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Feb 5 12:24:09 2016 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Feb 5 12:25:48 2016 -0600

----------------------------------------------------------------------

----------------------------------------------------------------------



[26/32] hbase git commit: Revert "HBASE-15122 Servlets generate XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER findbugs warnings (Samir Ahmic)" Revert mistaken commit.

Posted by sy...@apache.org.
Revert "HBASE-15122 Servlets generate XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER findbugs warnings (Samir Ahmic)"
Revert mistaken commit.

This reverts commit efc7a0d34749091d8efa623e7424956b72d3bb59.


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

Branch: refs/heads/hbase-12439
Commit: d82ae421262b5adc242fd7e04d1c774cdb43e639
Parents: 4e44f4f
Author: stack <st...@apache.org>
Authored: Sat Feb 6 05:17:29 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sat Feb 6 05:17:29 2016 -0800

----------------------------------------------------------------------
 .../src/main/resources/supplemental-models.xml  |  36 --
 hbase-server/pom.xml                            |  11 -
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |   8 +-
 .../src/main/resources/ESAPI.properties         | 431 -------------------
 .../hbase/http/jmx/TestJMXJsonServlet.java      |   6 -
 .../src/test/resources/ESAPI.properties         | 431 -------------------
 pom.xml                                         |   1 -
 7 files changed, 1 insertion(+), 923 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-resource-bundle/src/main/resources/supplemental-models.xml
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/supplemental-models.xml b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
index 764667c..2f94226 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -61,24 +61,6 @@ under the License.
       </licenses>
     </project>
   </supplement>
-  <supplement>
-    <project>
-      <groupId>commons-beanutils</groupId>
-      <artifactId>commons-beanutils-core</artifactId>
-
-      <organization>
-        <name>The Apache Software Foundation</name>
-        <url>http://www.apache.org/</url>
-      </organization>
-      <licenses>
-        <license>
-          <name>Apache Software License, Version 2.0</name>
-          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-          <distribution>repo</distribution>
-        </license>
-      </licenses>
-    </project>
-  </supplement>
 <!-- Artifacts with ambiguously named licenses in POM -->
   <supplement>
     <project>
@@ -1213,22 +1195,4 @@ Copyright (c) 2007-2011 The JRuby project
       </licenses>
     </project>
   </supplement>
-  <supplement>
-    <project>
-      <groupId>xalan</groupId>
-      <artifactId>xalan</artifactId>
-
-      <organization>
-        <name>The Apache Software Foundation</name>
-        <url>http://www.apache.org/</url>
-      </organization>
-      <licenses>
-        <license>
-          <name>The Apache Software License, Version 2.0</name>
-          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-          <distribution>repo</distribution>
-        </license>
-      </licenses>
-    </project>
-  </supplement>
 </supplementalDataModels>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index d5f1e30..3c25094 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -561,17 +561,6 @@
       <artifactId>bcprov-jdk16</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.owasp.esapi</groupId>
-      <artifactId>esapi</artifactId>
-      <version>2.1.0</version>
-      <exclusions>
-        <exclusion>
-          <artifactId>xercesImpl</artifactId>
-          <groupId>xerces</groupId>
-        </exclusion>
-      </exclusions>
-    </dependency>
   </dependencies>
   <profiles>
     <!-- Needs to make the profile in apache parent pom -->

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index 14a19f6..45c2c15 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -35,7 +35,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.http.HttpServer;
 import org.apache.hadoop.hbase.util.JSONBean;
-import org.owasp.esapi.ESAPI;
 
 /*
  * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
@@ -168,7 +167,7 @@ public class JMXJsonServlet extends HttpServlet {
         jsonpcb = request.getParameter(CALLBACK_PARAM);
         if (jsonpcb != null) {
           response.setContentType("application/javascript; charset=utf8");
-          writer.write(encodeJS(jsonpcb) + "(");
+          writer.write(jsonpcb + "(");
         } else {
           response.setContentType("application/json; charset=utf8");
         }
@@ -221,9 +220,4 @@ public class JMXJsonServlet extends HttpServlet {
       response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
     }
   }
-
-  private String encodeJS(String inputStr) {
-    return ESAPI.encoder().encodeForJavaScript(inputStr);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-server/src/main/resources/ESAPI.properties
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/ESAPI.properties b/hbase-server/src/main/resources/ESAPI.properties
deleted file mode 100644
index 9074001..0000000
--- a/hbase-server/src/main/resources/ESAPI.properties
+++ /dev/null
@@ -1,431 +0,0 @@
-#
-# OWASP Enterprise Security API (ESAPI) Properties file -- PRODUCTION Version
-#
-# This file is part of the Open Web Application Security Project (OWASP)
-# Enterprise Security API (ESAPI) project. For details, please see
-# http://www.owasp.org/index.php/ESAPI.
-#
-# Copyright (c) 2008,2009 - The OWASP Foundation
-#
-# DISCUSS: This may cause a major backwards compatibility issue, etc. but
-#           from a name space perspective, we probably should have prefaced
-#           all the property names with ESAPI or at least OWASP. Otherwise
-#           there could be problems is someone loads this properties file into
-#           the System properties.  We could also put this file into the
-#           esapi.jar file (perhaps as a ResourceBundle) and then allow an external
-#           ESAPI properties be defined that would overwrite these defaults.
-#           That keeps the application's properties relatively simple as usually
-#           they will only want to override a few properties. If looks like we
-#           already support multiple override levels of this in the
-#           DefaultSecurityConfiguration class, but I'm suggesting placing the
-#           defaults in the esapi.jar itself. That way, if the jar is signed,
-#           we could detect if those properties had been tampered with. (The
-#           code to check the jar signatures is pretty simple... maybe 70-90 LOC,
-#           but off course there is an execution penalty (similar to the way
-#           that the separate sunjce.jar used to be when a class from it was
-#           first loaded). Thoughts?
-###############################################################################
-#
-# WARNING: Operating system protection should be used to lock down the .esapi
-# resources directory and all the files inside and all the directories all the
-# way up to the root directory of the file system.  Note that if you are using
-# file-based implementations, that some files may need to be read-write as they
-# get updated dynamically.
-#
-# Before using, be sure to update the MasterKey and MasterSalt as described below.
-# N.B.: If you had stored data that you have previously encrypted with ESAPI 1.4,
-#        you *must* FIRST decrypt it using ESAPI 1.4 and then (if so desired)
-#        re-encrypt it with ESAPI 2.0. If you fail to do this, you will NOT be
-#        able to decrypt your data with ESAPI 2.0.
-#
-#        YOU HAVE BEEN WARNED!!! More details are in the ESAPI 2.0 Release Notes.
-#
-#===========================================================================
-# ESAPI Configuration
-#
-# If true, then print all the ESAPI properties set here when they are loaded.
-# If false, they are not printed. Useful to reduce output when running JUnit tests.
-# If you need to troubleshoot a properties related problem, turning this on may help.
-# This is 'false' in the src/test/resources/.esapi version. It is 'true' by
-# default for reasons of backward compatibility with earlier ESAPI versions.
-ESAPI.printProperties=true
-
-# ESAPI is designed to be easily extensible. You can use the reference implementation
-# or implement your own providers to take advantage of your enterprise's security
-# infrastructure. The functions in ESAPI are referenced using the ESAPI locator, like:
-#
-#    String ciphertext =
-#        ESAPI.encryptor().encrypt("Secret message");   // Deprecated in 2.0
-#    CipherText cipherText =
-#        ESAPI.encryptor().encrypt(new PlainText("Secret message")); // Preferred
-#
-# Below you can specify the classname for the provider that you wish to use in your
-# application. The only requirement is that it implement the appropriate ESAPI interface.
-# This allows you to switch security implementations in the future without rewriting the
-# entire application.
-#
-# ExperimentalAccessController requires ESAPI-AccessControlPolicy.xml in .esapi directory
-ESAPI.AccessControl=org.owasp.esapi.reference.DefaultAccessController
-# FileBasedAuthenticator requires users.txt file in .esapi directory
-ESAPI.Authenticator=org.owasp.esapi.reference.FileBasedAuthenticator
-ESAPI.Encoder=org.owasp.esapi.reference.DefaultEncoder
-ESAPI.Encryptor=org.owasp.esapi.reference.crypto.JavaEncryptor
-
-ESAPI.Executor=org.owasp.esapi.reference.DefaultExecutor
-ESAPI.HTTPUtilities=org.owasp.esapi.reference.DefaultHTTPUtilities
-ESAPI.IntrusionDetector=org.owasp.esapi.reference.DefaultIntrusionDetector
-# Log4JFactory Requires log4j.xml or log4j.properties in classpath - http://www.laliluna.de/log4j-tutorial.html
-ESAPI.Logger=org.owasp.esapi.reference.Log4JLogFactory
-#ESAPI.Logger=org.owasp.esapi.reference.JavaLogFactory
-ESAPI.Randomizer=org.owasp.esapi.reference.DefaultRandomizer
-ESAPI.Validator=org.owasp.esapi.reference.DefaultValidator
-
-#===========================================================================
-# ESAPI Authenticator
-#
-Authenticator.AllowedLoginAttempts=3
-Authenticator.MaxOldPasswordHashes=13
-Authenticator.UsernameParameterName=username
-Authenticator.PasswordParameterName=password
-# RememberTokenDuration (in days)
-Authenticator.RememberTokenDuration=14
-# Session Timeouts (in minutes)
-Authenticator.IdleTimeoutDuration=20
-Authenticator.AbsoluteTimeoutDuration=120
-
-#===========================================================================
-# ESAPI Encoder
-#
-# ESAPI canonicalizes input before validation to prevent bypassing filters with encoded attacks.
-# Failure to canonicalize input is a very common mistake when implementing validation schemes.
-# Canonicalization is automatic when using the ESAPI Validator, but you can also use the
-# following code to canonicalize data.
-#
-#      ESAPI.Encoder().canonicalize( "%22hello world&#x22;" );
-#
-# Multiple encoding is when a single encoding format is applied multiple times, multiple
-# different encoding formats are applied, or when multiple formats are nested. Allowing
-# multiple encoding is strongly discouraged.
-Encoder.AllowMultipleEncoding=false
-#
-# The default list of codecs to apply when canonicalizing untrusted data. The list should include the codecs
-# for all downstream interpreters or decoders. For example, if the data is likely to end up in a URL, HTML, or
-# inside JavaScript, then the list of codecs below is appropriate. The order of the list is not terribly important.
-Encoder.DefaultCodecList=HTMLEntityCodec,PercentCodec,JavaScriptCodec
-
-
-#===========================================================================
-# ESAPI Encryption
-#
-# The ESAPI Encryptor provides basic cryptographic functions with a simplified API.
-# To get started, generate a new key using java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
-# There is not currently any support for key rotation, so be careful when changing your key and salt as it
-# will invalidate all signed, encrypted, and hashed data.
-#
-# WARNING: Not all combinations of algorithms and key lengths are supported.
-# If you choose to use a key length greater than 128, you MUST download the
-# unlimited strength policy files and install in the lib directory of your JRE/JDK.
-# See http://java.sun.com/javase/downloads/index.jsp for more information.
-#
-# Backward compatibility with ESAPI Java 1.4 is supported by the two deprecated API
-# methods, Encryptor.encrypt(String) and Encryptor.decrypt(String). However, whenever
-# possible, these methods should be avoided as they use ECB cipher mode, which in almost
-# all circumstances a poor choice because of it's weakness. CBC cipher mode is the default
-# for the new Encryptor encrypt / decrypt methods for ESAPI Java 2.0.  In general, you
-# should only use this compatibility setting if you have persistent data encrypted with
-# version 1.4 and even then, you should ONLY set this compatibility mode UNTIL
-# you have decrypted all of your old encrypted data and then re-encrypted it with
-# ESAPI 2.0 using CBC mode. If you have some reason to mix the deprecated 1.4 mode
-# with the new 2.0 methods, make sure that you use the same cipher algorithm for both
-# (256-bit AES was the default for 1.4; 128-bit is the default for 2.0; see below for
-# more details.) Otherwise, you will have to use the new 2.0 encrypt / decrypt methods
-# where you can specify a SecretKey. (Note that if you are using the 256-bit AES,
-# that requires downloading the special jurisdiction policy files mentioned above.)
-#
-#        ***** IMPORTANT: Do NOT forget to replace these with your own values! *****
-# To calculate these values, you can run:
-#        java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
-#
-Encryptor.MasterKey=
-Encryptor.MasterSalt=
-
-# Provides the default JCE provider that ESAPI will "prefer" for its symmetric
-# encryption and hashing. (That is it will look to this provider first, but it
-# will defer to other providers if the requested algorithm is not implemented
-# by this provider.) If left unset, ESAPI will just use your Java VM's current
-# preferred JCE provider, which is generally set in the file
-# "$JAVA_HOME/jre/lib/security/java.security".
-#
-# The main intent of this is to allow ESAPI symmetric encryption to be
-# used with a FIPS 140-2 compliant crypto-module. For details, see the section
-# "Using ESAPI Symmetric Encryption with FIPS 140-2 Cryptographic Modules" in
-# the ESAPI 2.0 Symmetric Encryption User Guide, at:
-# http://owasp-esapi-java.googlecode.com/svn/trunk/documentation/esapi4java-core-2.0-symmetric-crypto-user-guide.html
-# However, this property also allows you to easily use an alternate JCE provider
-# such as "Bouncy Castle" without having to make changes to "java.security".
-# See Javadoc for SecurityProviderLoader for further details. If you wish to use
-# a provider that is not known to SecurityProviderLoader, you may specify the
-# fully-qualified class name of the JCE provider class that implements
-# java.security.Provider. If the name contains a '.', this is interpreted as
-# a fully-qualified class name that implements java.security.Provider.
-#
-# NOTE: Setting this property has the side-effect of changing it in your application
-#       as well, so if you are using JCE in your application directly rather than
-#       through ESAPI (you wouldn't do that, would you? ;-), it will change the
-#       preferred JCE provider there as well.
-#
-# Default: Keeps the JCE provider set to whatever JVM sets it to.
-Encryptor.PreferredJCEProvider=
-
-# AES is the most widely used and strongest encryption algorithm. This
-# should agree with your Encryptor.CipherTransformation property.
-# By default, ESAPI Java 1.4 uses "PBEWithMD5AndDES" and which is
-# very weak. It is essentially a password-based encryption key, hashed
-# with MD5 around 1K times and then encrypted with the weak DES algorithm
-# (56-bits) using ECB mode and an unspecified padding (it is
-# JCE provider specific, but most likely "NoPadding"). However, 2.0 uses
-# "AES/CBC/PKCSPadding". If you want to change these, change them here.
-# Warning: This property does not control the default reference implementation for
-#           ESAPI 2.0 using JavaEncryptor. Also, this property will be dropped
-#           in the future.
-# @deprecated
-Encryptor.EncryptionAlgorithm=AES
-#        For ESAPI Java 2.0 - New encrypt / decrypt methods use this.
-Encryptor.CipherTransformation=AES/CBC/PKCS5Padding
-
-# Applies to ESAPI 2.0 and later only!
-# Comma-separated list of cipher modes that provide *BOTH*
-# confidentiality *AND* message authenticity. (NIST refers to such cipher
-# modes as "combined modes" so that's what we shall call them.) If any of these
-# cipher modes are used then no MAC is calculated and stored
-# in the CipherText upon encryption. Likewise, if one of these
-# cipher modes is used with decryption, no attempt will be made
-# to validate the MAC contained in the CipherText object regardless
-# of whether it contains one or not. Since the expectation is that
-# these cipher modes support support message authenticity already,
-# injecting a MAC in the CipherText object would be at best redundant.
-#
-# Note that as of JDK 1.5, the SunJCE provider does not support *any*
-# of these cipher modes. Of these listed, only GCM and CCM are currently
-# NIST approved. YMMV for other JCE providers. E.g., Bouncy Castle supports
-# GCM and CCM with "NoPadding" mode, but not with "PKCS5Padding" or other
-# padding modes.
-Encryptor.cipher_modes.combined_modes=GCM,CCM,IAPM,EAX,OCB,CWC
-
-# Applies to ESAPI 2.0 and later only!
-# Additional cipher modes allowed for ESAPI 2.0 encryption. These
-# cipher modes are in _addition_ to those specified by the property
-# 'Encryptor.cipher_modes.combined_modes'.
-# Note: We will add support for streaming modes like CFB & OFB once
-# we add support for 'specified' to the property 'Encryptor.ChooseIVMethod'
-# (probably in ESAPI 2.1).
-# DISCUSS: Better name?
-Encryptor.cipher_modes.additional_allowed=CBC
-
-# 128-bit is almost always sufficient and appears to be more resistant to
-# related key attacks than is 256-bit AES. Use '_' to use default key size
-# for cipher algorithms (where it makes sense because the algorithm supports
-# a variable key size). Key length must agree to what's provided as the
-# cipher transformation, otherwise this will be ignored after logging a
-# warning.
-#
-# NOTE: This is what applies BOTH ESAPI 1.4 and 2.0. See warning above about mixing!
-Encryptor.EncryptionKeyLength=128
-
-# Because 2.0 uses CBC mode by default, it requires an initialization vector (IV).
-# (All cipher modes except ECB require an IV.) There are two choices: we can either
-# use a fixed IV known to both parties or allow ESAPI to choose a random IV. While
-# the IV does not need to be hidden from adversaries, it is important that the
-# adversary not be allowed to choose it. Also, random IVs are generally much more
-# secure than fixed IVs. (In fact, it is essential that feed-back cipher modes
-# such as CFB and OFB use a different IV for each encryption with a given key so
-# in such cases, random IVs are much preferred. By default, ESAPI 2.0 uses random
-# IVs. If you wish to use 'fixed' IVs, set 'Encryptor.ChooseIVMethod=fixed' and
-# uncomment the Encryptor.fixedIV.
-#
-# Valid values:        random|fixed|specified        'specified' not yet implemented; planned for 2.1
-Encryptor.ChooseIVMethod=random
-# If you choose to use a fixed IV, then you must place a fixed IV here that
-# is known to all others who are sharing your secret key. The format should
-# be a hex string that is the same length as the cipher block size for the
-# cipher algorithm that you are using. The following is an example for AES
-# from an AES test vector for AES-128/CBC as described in:
-# NIST Special Publication 800-38A (2001 Edition)
-# "Recommendation for Block Cipher Modes of Operation".
-# (Note that the block size for AES is 16 bytes == 128 bits.)
-#
-Encryptor.fixedIV=0x000102030405060708090a0b0c0d0e0f
-
-# Whether or not CipherText should use a message authentication code (MAC) with it.
-# This prevents an adversary from altering the IV as well as allowing a more
-# fool-proof way of determining the decryption failed because of an incorrect
-# key being supplied. This refers to the "separate" MAC calculated and stored
-# in CipherText, not part of any MAC that is calculated as a result of a
-# "combined mode" cipher mode.
-#
-# If you are using ESAPI with a FIPS 140-2 cryptographic module, you *must* also
-# set this property to false.
-Encryptor.CipherText.useMAC=true
-
-# Whether or not the PlainText object may be overwritten and then marked
-# eligible for garbage collection. If not set, this is still treated as 'true'.
-Encryptor.PlainText.overwrite=true
-
-# Do not use DES except in a legacy situations. 56-bit is way too small key size.
-#Encryptor.EncryptionKeyLength=56
-#Encryptor.EncryptionAlgorithm=DES
-
-# TripleDES is considered strong enough for most purposes.
-#    Note:    There is also a 112-bit version of DESede. Using the 168-bit version
-#            requires downloading the special jurisdiction policy from Sun.
-#Encryptor.EncryptionKeyLength=168
-#Encryptor.EncryptionAlgorithm=DESede
-
-Encryptor.HashAlgorithm=SHA-512
-Encryptor.HashIterations=1024
-Encryptor.DigitalSignatureAlgorithm=SHA1withDSA
-Encryptor.DigitalSignatureKeyLength=1024
-Encryptor.RandomAlgorithm=SHA1PRNG
-Encryptor.CharacterEncoding=UTF-8
-
-
-#===========================================================================
-# ESAPI HttpUtilties
-#
-# The HttpUtilities provide basic protections to HTTP requests and responses. Primarily these methods
-# protect against malicious data from attackers, such as unprintable characters, escaped characters,
-# and other simple attacks. The HttpUtilities also provides utility methods for dealing with cookies,
-# headers, and CSRF tokens.
-#
-# Default file upload location (remember to escape backslashes with \\)
-HttpUtilities.UploadDir=C:\\ESAPI\\testUpload
-HttpUtilities.UploadTempDir=C:\\temp
-# Force flags on cookies, if you use HttpUtilities to set cookies
-HttpUtilities.ForceHttpOnlySession=false
-HttpUtilities.ForceSecureSession=false
-HttpUtilities.ForceHttpOnlyCookies=true
-HttpUtilities.ForceSecureCookies=true
-# Maximum size of HTTP headers
-HttpUtilities.MaxHeaderSize=4096
-# File upload configuration
-HttpUtilities.ApprovedUploadExtensions=.zip,.pdf,.doc,.docx,.ppt,.pptx,.tar,.gz,.tgz,.rar,.war,.jar,.ear,.xls,.rtf,.properties,.java,.class,.txt,.xml,.jsp,.jsf,.exe,.dll
-HttpUtilities.MaxUploadFileBytes=500000000
-# Using UTF-8 throughout your stack is highly recommended. That includes your database driver,
-# container, and any other technologies you may be using. Failure to do this may expose you
-# to Unicode transcoding injection attacks. Use of UTF-8 does not hinder internationalization.
-HttpUtilities.ResponseContentType=text/html; charset=UTF-8
-
-
-
-#===========================================================================
-# ESAPI Executor
-# CHECKME - Not sure what this is used for, but surely it should be made OS independent.
-Executor.WorkingDirectory=C:\\Windows\\Temp
-Executor.ApprovedExecutables=C:\\Windows\\System32\\cmd.exe,C:\\Windows\\System32\\runas.exe
-
-
-#===========================================================================
-# ESAPI Logging
-# Set the application name if these logs are combined with other applications
-Logger.ApplicationName=ExampleApplication
-# If you use an HTML log viewer that does not properly HTML escape log data, you can set LogEncodingRequired to true
-Logger.LogEncodingRequired=false
-# Determines whether ESAPI should log the application name. This might be clutter in some single-server/single-app environments.
-Logger.LogApplicationName=true
-# Determines whether ESAPI should log the server IP and port. This might be clutter in some single-server environments.
-Logger.LogServerIP=true
-# LogFileName, the name of the logging file. Provide a full directory path (e.g., C:\\ESAPI\\ESAPI_logging_file) if you
-# want to place it in a specific directory.
-Logger.LogFileName=ESAPI_logging_file
-# MaxLogFileSize, the max size (in bytes) of a single log file before it cuts over to a new one (default is 10,000,000)
-Logger.MaxLogFileSize=10000000
-
-
-#===========================================================================
-# ESAPI Intrusion Detection
-#
-# Each event has a base to which .count, .interval, and .action are added
-# The IntrusionException will fire if we receive "count" events within "interval" seconds
-# The IntrusionDetector is configurable to take the following actions: log, logout, and disable
-#  (multiple actions separated by commas are allowed e.g. event.test.actions=log,disable
-#
-# Custom Events
-# Names must start with "event." as the base
-# Use IntrusionDetector.addEvent( "test" ) in your code to trigger "event.test" here
-# You can also disable intrusion detection completely by changing
-# the following parameter to true
-#
-IntrusionDetector.Disable=false
-#
-IntrusionDetector.event.test.count=2
-IntrusionDetector.event.test.interval=10
-IntrusionDetector.event.test.actions=disable,log
-
-# Exception Events
-# All EnterpriseSecurityExceptions are registered automatically
-# Call IntrusionDetector.getInstance().addException(e) for Exceptions that do not extend EnterpriseSecurityException
-# Use the fully qualified classname of the exception as the base
-
-# any intrusion is an attack
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.count=1
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.interval=1
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.actions=log,disable,logout
-
-# for test purposes
-# CHECKME: Shouldn't there be something in the property name itself that designates
-#           that these are for testing???
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.count=10
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.interval=5
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.actions=log,disable,logout
-
-# rapid validation errors indicate scans or attacks in progress
-# org.owasp.esapi.errors.ValidationException.count=10
-# org.owasp.esapi.errors.ValidationException.interval=10
-# org.owasp.esapi.errors.ValidationException.actions=log,logout
-
-# sessions jumping between hosts indicates session hijacking
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.count=2
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.interval=10
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.actions=log,logout
-
-
-#===========================================================================
-# ESAPI Validation
-#
-# The ESAPI Validator works on regular expressions with defined names. You can define names
-# either here, or you may define application specific patterns in a separate file defined below.
-# This allows enterprises to specify both organizational standards as well as application specific
-# validation rules.
-#
-Validator.ConfigurationFile=validation.properties
-
-# Validators used by ESAPI
-Validator.AccountName=^[a-zA-Z0-9]{3,20}$
-Validator.SystemCommand=^[a-zA-Z\\-\\/]{1,64}$
-Validator.RoleName=^[a-z]{1,20}$
-
-#the word TEST below should be changed to your application
-#name - only relative URL's are supported
-Validator.Redirect=^\\/test.*$
-
-# Global HTTP Validation Rules
-# Values with Base64 encoded data (e.g. encrypted state) will need at least [a-zA-Z0-9\/+=]
-Validator.HTTPScheme=^(http|https)$
-Validator.HTTPServerName=^[a-zA-Z0-9_.\\-]*$
-Validator.HTTPParameterName=^[a-zA-Z0-9_]{1,32}$
-Validator.HTTPParameterValue=^[a-zA-Z0-9.\\-\\/+=_ ]*$
-Validator.HTTPCookieName=^[a-zA-Z0-9\\-_]{1,32}$
-Validator.HTTPCookieValue=^[a-zA-Z0-9\\-\\/+=_ ]*$
-Validator.HTTPHeaderName=^[a-zA-Z0-9\\-_]{1,32}$
-Validator.HTTPHeaderValue=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
-Validator.HTTPContextPath=^[a-zA-Z0-9.\\-\\/_]*$
-Validator.HTTPServletPath=^[a-zA-Z0-9.\\-\\/_]*$
-Validator.HTTPPath=^[a-zA-Z0-9.\\-_]*$
-Validator.HTTPQueryString=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ %]*$
-Validator.HTTPURI=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
-Validator.HTTPURL=^.*$
-Validator.HTTPJSESSIONID=^[A-Z0-9]{10,30}$
-
-# Validation of file related input
-Validator.FileName=^[a-zA-Z0-9!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$
-Validator.DirectoryName=^[a-zA-Z0-9:/\\\\!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
index baeaf89..031ddce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
@@ -105,11 +105,5 @@ public class TestJMXJsonServlet extends HttpServerFunctionalTest {
     assertReFind("\"committed\"\\s*:", result);
     assertReFind("\\}\\);$", result);
 
-    // test to get XSS JSONP result
-    result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Memory&callback=<script>alert('hello')</script>"));
-    LOG.info("/jmx?qry=java.lang:type=Memory&callback=<script>alert('hello')</script> RESULT: "+result);
-    assertTrue(!result.contains("<script>"));
-
-
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/hbase-server/src/test/resources/ESAPI.properties
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/ESAPI.properties b/hbase-server/src/test/resources/ESAPI.properties
deleted file mode 100644
index 9074001..0000000
--- a/hbase-server/src/test/resources/ESAPI.properties
+++ /dev/null
@@ -1,431 +0,0 @@
-#
-# OWASP Enterprise Security API (ESAPI) Properties file -- PRODUCTION Version
-#
-# This file is part of the Open Web Application Security Project (OWASP)
-# Enterprise Security API (ESAPI) project. For details, please see
-# http://www.owasp.org/index.php/ESAPI.
-#
-# Copyright (c) 2008,2009 - The OWASP Foundation
-#
-# DISCUSS: This may cause a major backwards compatibility issue, etc. but
-#           from a name space perspective, we probably should have prefaced
-#           all the property names with ESAPI or at least OWASP. Otherwise
-#           there could be problems is someone loads this properties file into
-#           the System properties.  We could also put this file into the
-#           esapi.jar file (perhaps as a ResourceBundle) and then allow an external
-#           ESAPI properties be defined that would overwrite these defaults.
-#           That keeps the application's properties relatively simple as usually
-#           they will only want to override a few properties. If looks like we
-#           already support multiple override levels of this in the
-#           DefaultSecurityConfiguration class, but I'm suggesting placing the
-#           defaults in the esapi.jar itself. That way, if the jar is signed,
-#           we could detect if those properties had been tampered with. (The
-#           code to check the jar signatures is pretty simple... maybe 70-90 LOC,
-#           but off course there is an execution penalty (similar to the way
-#           that the separate sunjce.jar used to be when a class from it was
-#           first loaded). Thoughts?
-###############################################################################
-#
-# WARNING: Operating system protection should be used to lock down the .esapi
-# resources directory and all the files inside and all the directories all the
-# way up to the root directory of the file system.  Note that if you are using
-# file-based implementations, that some files may need to be read-write as they
-# get updated dynamically.
-#
-# Before using, be sure to update the MasterKey and MasterSalt as described below.
-# N.B.: If you had stored data that you have previously encrypted with ESAPI 1.4,
-#        you *must* FIRST decrypt it using ESAPI 1.4 and then (if so desired)
-#        re-encrypt it with ESAPI 2.0. If you fail to do this, you will NOT be
-#        able to decrypt your data with ESAPI 2.0.
-#
-#        YOU HAVE BEEN WARNED!!! More details are in the ESAPI 2.0 Release Notes.
-#
-#===========================================================================
-# ESAPI Configuration
-#
-# If true, then print all the ESAPI properties set here when they are loaded.
-# If false, they are not printed. Useful to reduce output when running JUnit tests.
-# If you need to troubleshoot a properties related problem, turning this on may help.
-# This is 'false' in the src/test/resources/.esapi version. It is 'true' by
-# default for reasons of backward compatibility with earlier ESAPI versions.
-ESAPI.printProperties=true
-
-# ESAPI is designed to be easily extensible. You can use the reference implementation
-# or implement your own providers to take advantage of your enterprise's security
-# infrastructure. The functions in ESAPI are referenced using the ESAPI locator, like:
-#
-#    String ciphertext =
-#        ESAPI.encryptor().encrypt("Secret message");   // Deprecated in 2.0
-#    CipherText cipherText =
-#        ESAPI.encryptor().encrypt(new PlainText("Secret message")); // Preferred
-#
-# Below you can specify the classname for the provider that you wish to use in your
-# application. The only requirement is that it implement the appropriate ESAPI interface.
-# This allows you to switch security implementations in the future without rewriting the
-# entire application.
-#
-# ExperimentalAccessController requires ESAPI-AccessControlPolicy.xml in .esapi directory
-ESAPI.AccessControl=org.owasp.esapi.reference.DefaultAccessController
-# FileBasedAuthenticator requires users.txt file in .esapi directory
-ESAPI.Authenticator=org.owasp.esapi.reference.FileBasedAuthenticator
-ESAPI.Encoder=org.owasp.esapi.reference.DefaultEncoder
-ESAPI.Encryptor=org.owasp.esapi.reference.crypto.JavaEncryptor
-
-ESAPI.Executor=org.owasp.esapi.reference.DefaultExecutor
-ESAPI.HTTPUtilities=org.owasp.esapi.reference.DefaultHTTPUtilities
-ESAPI.IntrusionDetector=org.owasp.esapi.reference.DefaultIntrusionDetector
-# Log4JFactory Requires log4j.xml or log4j.properties in classpath - http://www.laliluna.de/log4j-tutorial.html
-ESAPI.Logger=org.owasp.esapi.reference.Log4JLogFactory
-#ESAPI.Logger=org.owasp.esapi.reference.JavaLogFactory
-ESAPI.Randomizer=org.owasp.esapi.reference.DefaultRandomizer
-ESAPI.Validator=org.owasp.esapi.reference.DefaultValidator
-
-#===========================================================================
-# ESAPI Authenticator
-#
-Authenticator.AllowedLoginAttempts=3
-Authenticator.MaxOldPasswordHashes=13
-Authenticator.UsernameParameterName=username
-Authenticator.PasswordParameterName=password
-# RememberTokenDuration (in days)
-Authenticator.RememberTokenDuration=14
-# Session Timeouts (in minutes)
-Authenticator.IdleTimeoutDuration=20
-Authenticator.AbsoluteTimeoutDuration=120
-
-#===========================================================================
-# ESAPI Encoder
-#
-# ESAPI canonicalizes input before validation to prevent bypassing filters with encoded attacks.
-# Failure to canonicalize input is a very common mistake when implementing validation schemes.
-# Canonicalization is automatic when using the ESAPI Validator, but you can also use the
-# following code to canonicalize data.
-#
-#      ESAPI.Encoder().canonicalize( "%22hello world&#x22;" );
-#
-# Multiple encoding is when a single encoding format is applied multiple times, multiple
-# different encoding formats are applied, or when multiple formats are nested. Allowing
-# multiple encoding is strongly discouraged.
-Encoder.AllowMultipleEncoding=false
-#
-# The default list of codecs to apply when canonicalizing untrusted data. The list should include the codecs
-# for all downstream interpreters or decoders. For example, if the data is likely to end up in a URL, HTML, or
-# inside JavaScript, then the list of codecs below is appropriate. The order of the list is not terribly important.
-Encoder.DefaultCodecList=HTMLEntityCodec,PercentCodec,JavaScriptCodec
-
-
-#===========================================================================
-# ESAPI Encryption
-#
-# The ESAPI Encryptor provides basic cryptographic functions with a simplified API.
-# To get started, generate a new key using java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
-# There is not currently any support for key rotation, so be careful when changing your key and salt as it
-# will invalidate all signed, encrypted, and hashed data.
-#
-# WARNING: Not all combinations of algorithms and key lengths are supported.
-# If you choose to use a key length greater than 128, you MUST download the
-# unlimited strength policy files and install in the lib directory of your JRE/JDK.
-# See http://java.sun.com/javase/downloads/index.jsp for more information.
-#
-# Backward compatibility with ESAPI Java 1.4 is supported by the two deprecated API
-# methods, Encryptor.encrypt(String) and Encryptor.decrypt(String). However, whenever
-# possible, these methods should be avoided as they use ECB cipher mode, which in almost
-# all circumstances a poor choice because of it's weakness. CBC cipher mode is the default
-# for the new Encryptor encrypt / decrypt methods for ESAPI Java 2.0.  In general, you
-# should only use this compatibility setting if you have persistent data encrypted with
-# version 1.4 and even then, you should ONLY set this compatibility mode UNTIL
-# you have decrypted all of your old encrypted data and then re-encrypted it with
-# ESAPI 2.0 using CBC mode. If you have some reason to mix the deprecated 1.4 mode
-# with the new 2.0 methods, make sure that you use the same cipher algorithm for both
-# (256-bit AES was the default for 1.4; 128-bit is the default for 2.0; see below for
-# more details.) Otherwise, you will have to use the new 2.0 encrypt / decrypt methods
-# where you can specify a SecretKey. (Note that if you are using the 256-bit AES,
-# that requires downloading the special jurisdiction policy files mentioned above.)
-#
-#        ***** IMPORTANT: Do NOT forget to replace these with your own values! *****
-# To calculate these values, you can run:
-#        java -classpath esapi.jar org.owasp.esapi.reference.crypto.JavaEncryptor
-#
-Encryptor.MasterKey=
-Encryptor.MasterSalt=
-
-# Provides the default JCE provider that ESAPI will "prefer" for its symmetric
-# encryption and hashing. (That is it will look to this provider first, but it
-# will defer to other providers if the requested algorithm is not implemented
-# by this provider.) If left unset, ESAPI will just use your Java VM's current
-# preferred JCE provider, which is generally set in the file
-# "$JAVA_HOME/jre/lib/security/java.security".
-#
-# The main intent of this is to allow ESAPI symmetric encryption to be
-# used with a FIPS 140-2 compliant crypto-module. For details, see the section
-# "Using ESAPI Symmetric Encryption with FIPS 140-2 Cryptographic Modules" in
-# the ESAPI 2.0 Symmetric Encryption User Guide, at:
-# http://owasp-esapi-java.googlecode.com/svn/trunk/documentation/esapi4java-core-2.0-symmetric-crypto-user-guide.html
-# However, this property also allows you to easily use an alternate JCE provider
-# such as "Bouncy Castle" without having to make changes to "java.security".
-# See Javadoc for SecurityProviderLoader for further details. If you wish to use
-# a provider that is not known to SecurityProviderLoader, you may specify the
-# fully-qualified class name of the JCE provider class that implements
-# java.security.Provider. If the name contains a '.', this is interpreted as
-# a fully-qualified class name that implements java.security.Provider.
-#
-# NOTE: Setting this property has the side-effect of changing it in your application
-#       as well, so if you are using JCE in your application directly rather than
-#       through ESAPI (you wouldn't do that, would you? ;-), it will change the
-#       preferred JCE provider there as well.
-#
-# Default: Keeps the JCE provider set to whatever JVM sets it to.
-Encryptor.PreferredJCEProvider=
-
-# AES is the most widely used and strongest encryption algorithm. This
-# should agree with your Encryptor.CipherTransformation property.
-# By default, ESAPI Java 1.4 uses "PBEWithMD5AndDES" and which is
-# very weak. It is essentially a password-based encryption key, hashed
-# with MD5 around 1K times and then encrypted with the weak DES algorithm
-# (56-bits) using ECB mode and an unspecified padding (it is
-# JCE provider specific, but most likely "NoPadding"). However, 2.0 uses
-# "AES/CBC/PKCSPadding". If you want to change these, change them here.
-# Warning: This property does not control the default reference implementation for
-#           ESAPI 2.0 using JavaEncryptor. Also, this property will be dropped
-#           in the future.
-# @deprecated
-Encryptor.EncryptionAlgorithm=AES
-#        For ESAPI Java 2.0 - New encrypt / decrypt methods use this.
-Encryptor.CipherTransformation=AES/CBC/PKCS5Padding
-
-# Applies to ESAPI 2.0 and later only!
-# Comma-separated list of cipher modes that provide *BOTH*
-# confidentiality *AND* message authenticity. (NIST refers to such cipher
-# modes as "combined modes" so that's what we shall call them.) If any of these
-# cipher modes are used then no MAC is calculated and stored
-# in the CipherText upon encryption. Likewise, if one of these
-# cipher modes is used with decryption, no attempt will be made
-# to validate the MAC contained in the CipherText object regardless
-# of whether it contains one or not. Since the expectation is that
-# these cipher modes support support message authenticity already,
-# injecting a MAC in the CipherText object would be at best redundant.
-#
-# Note that as of JDK 1.5, the SunJCE provider does not support *any*
-# of these cipher modes. Of these listed, only GCM and CCM are currently
-# NIST approved. YMMV for other JCE providers. E.g., Bouncy Castle supports
-# GCM and CCM with "NoPadding" mode, but not with "PKCS5Padding" or other
-# padding modes.
-Encryptor.cipher_modes.combined_modes=GCM,CCM,IAPM,EAX,OCB,CWC
-
-# Applies to ESAPI 2.0 and later only!
-# Additional cipher modes allowed for ESAPI 2.0 encryption. These
-# cipher modes are in _addition_ to those specified by the property
-# 'Encryptor.cipher_modes.combined_modes'.
-# Note: We will add support for streaming modes like CFB & OFB once
-# we add support for 'specified' to the property 'Encryptor.ChooseIVMethod'
-# (probably in ESAPI 2.1).
-# DISCUSS: Better name?
-Encryptor.cipher_modes.additional_allowed=CBC
-
-# 128-bit is almost always sufficient and appears to be more resistant to
-# related key attacks than is 256-bit AES. Use '_' to use default key size
-# for cipher algorithms (where it makes sense because the algorithm supports
-# a variable key size). Key length must agree to what's provided as the
-# cipher transformation, otherwise this will be ignored after logging a
-# warning.
-#
-# NOTE: This is what applies BOTH ESAPI 1.4 and 2.0. See warning above about mixing!
-Encryptor.EncryptionKeyLength=128
-
-# Because 2.0 uses CBC mode by default, it requires an initialization vector (IV).
-# (All cipher modes except ECB require an IV.) There are two choices: we can either
-# use a fixed IV known to both parties or allow ESAPI to choose a random IV. While
-# the IV does not need to be hidden from adversaries, it is important that the
-# adversary not be allowed to choose it. Also, random IVs are generally much more
-# secure than fixed IVs. (In fact, it is essential that feed-back cipher modes
-# such as CFB and OFB use a different IV for each encryption with a given key so
-# in such cases, random IVs are much preferred. By default, ESAPI 2.0 uses random
-# IVs. If you wish to use 'fixed' IVs, set 'Encryptor.ChooseIVMethod=fixed' and
-# uncomment the Encryptor.fixedIV.
-#
-# Valid values:        random|fixed|specified        'specified' not yet implemented; planned for 2.1
-Encryptor.ChooseIVMethod=random
-# If you choose to use a fixed IV, then you must place a fixed IV here that
-# is known to all others who are sharing your secret key. The format should
-# be a hex string that is the same length as the cipher block size for the
-# cipher algorithm that you are using. The following is an example for AES
-# from an AES test vector for AES-128/CBC as described in:
-# NIST Special Publication 800-38A (2001 Edition)
-# "Recommendation for Block Cipher Modes of Operation".
-# (Note that the block size for AES is 16 bytes == 128 bits.)
-#
-Encryptor.fixedIV=0x000102030405060708090a0b0c0d0e0f
-
-# Whether or not CipherText should use a message authentication code (MAC) with it.
-# This prevents an adversary from altering the IV as well as allowing a more
-# fool-proof way of determining the decryption failed because of an incorrect
-# key being supplied. This refers to the "separate" MAC calculated and stored
-# in CipherText, not part of any MAC that is calculated as a result of a
-# "combined mode" cipher mode.
-#
-# If you are using ESAPI with a FIPS 140-2 cryptographic module, you *must* also
-# set this property to false.
-Encryptor.CipherText.useMAC=true
-
-# Whether or not the PlainText object may be overwritten and then marked
-# eligible for garbage collection. If not set, this is still treated as 'true'.
-Encryptor.PlainText.overwrite=true
-
-# Do not use DES except in a legacy situations. 56-bit is way too small key size.
-#Encryptor.EncryptionKeyLength=56
-#Encryptor.EncryptionAlgorithm=DES
-
-# TripleDES is considered strong enough for most purposes.
-#    Note:    There is also a 112-bit version of DESede. Using the 168-bit version
-#            requires downloading the special jurisdiction policy from Sun.
-#Encryptor.EncryptionKeyLength=168
-#Encryptor.EncryptionAlgorithm=DESede
-
-Encryptor.HashAlgorithm=SHA-512
-Encryptor.HashIterations=1024
-Encryptor.DigitalSignatureAlgorithm=SHA1withDSA
-Encryptor.DigitalSignatureKeyLength=1024
-Encryptor.RandomAlgorithm=SHA1PRNG
-Encryptor.CharacterEncoding=UTF-8
-
-
-#===========================================================================
-# ESAPI HttpUtilties
-#
-# The HttpUtilities provide basic protections to HTTP requests and responses. Primarily these methods
-# protect against malicious data from attackers, such as unprintable characters, escaped characters,
-# and other simple attacks. The HttpUtilities also provides utility methods for dealing with cookies,
-# headers, and CSRF tokens.
-#
-# Default file upload location (remember to escape backslashes with \\)
-HttpUtilities.UploadDir=C:\\ESAPI\\testUpload
-HttpUtilities.UploadTempDir=C:\\temp
-# Force flags on cookies, if you use HttpUtilities to set cookies
-HttpUtilities.ForceHttpOnlySession=false
-HttpUtilities.ForceSecureSession=false
-HttpUtilities.ForceHttpOnlyCookies=true
-HttpUtilities.ForceSecureCookies=true
-# Maximum size of HTTP headers
-HttpUtilities.MaxHeaderSize=4096
-# File upload configuration
-HttpUtilities.ApprovedUploadExtensions=.zip,.pdf,.doc,.docx,.ppt,.pptx,.tar,.gz,.tgz,.rar,.war,.jar,.ear,.xls,.rtf,.properties,.java,.class,.txt,.xml,.jsp,.jsf,.exe,.dll
-HttpUtilities.MaxUploadFileBytes=500000000
-# Using UTF-8 throughout your stack is highly recommended. That includes your database driver,
-# container, and any other technologies you may be using. Failure to do this may expose you
-# to Unicode transcoding injection attacks. Use of UTF-8 does not hinder internationalization.
-HttpUtilities.ResponseContentType=text/html; charset=UTF-8
-
-
-
-#===========================================================================
-# ESAPI Executor
-# CHECKME - Not sure what this is used for, but surely it should be made OS independent.
-Executor.WorkingDirectory=C:\\Windows\\Temp
-Executor.ApprovedExecutables=C:\\Windows\\System32\\cmd.exe,C:\\Windows\\System32\\runas.exe
-
-
-#===========================================================================
-# ESAPI Logging
-# Set the application name if these logs are combined with other applications
-Logger.ApplicationName=ExampleApplication
-# If you use an HTML log viewer that does not properly HTML escape log data, you can set LogEncodingRequired to true
-Logger.LogEncodingRequired=false
-# Determines whether ESAPI should log the application name. This might be clutter in some single-server/single-app environments.
-Logger.LogApplicationName=true
-# Determines whether ESAPI should log the server IP and port. This might be clutter in some single-server environments.
-Logger.LogServerIP=true
-# LogFileName, the name of the logging file. Provide a full directory path (e.g., C:\\ESAPI\\ESAPI_logging_file) if you
-# want to place it in a specific directory.
-Logger.LogFileName=ESAPI_logging_file
-# MaxLogFileSize, the max size (in bytes) of a single log file before it cuts over to a new one (default is 10,000,000)
-Logger.MaxLogFileSize=10000000
-
-
-#===========================================================================
-# ESAPI Intrusion Detection
-#
-# Each event has a base to which .count, .interval, and .action are added
-# The IntrusionException will fire if we receive "count" events within "interval" seconds
-# The IntrusionDetector is configurable to take the following actions: log, logout, and disable
-#  (multiple actions separated by commas are allowed e.g. event.test.actions=log,disable
-#
-# Custom Events
-# Names must start with "event." as the base
-# Use IntrusionDetector.addEvent( "test" ) in your code to trigger "event.test" here
-# You can also disable intrusion detection completely by changing
-# the following parameter to true
-#
-IntrusionDetector.Disable=false
-#
-IntrusionDetector.event.test.count=2
-IntrusionDetector.event.test.interval=10
-IntrusionDetector.event.test.actions=disable,log
-
-# Exception Events
-# All EnterpriseSecurityExceptions are registered automatically
-# Call IntrusionDetector.getInstance().addException(e) for Exceptions that do not extend EnterpriseSecurityException
-# Use the fully qualified classname of the exception as the base
-
-# any intrusion is an attack
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.count=1
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.interval=1
-IntrusionDetector.org.owasp.esapi.errors.IntrusionException.actions=log,disable,logout
-
-# for test purposes
-# CHECKME: Shouldn't there be something in the property name itself that designates
-#           that these are for testing???
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.count=10
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.interval=5
-IntrusionDetector.org.owasp.esapi.errors.IntegrityException.actions=log,disable,logout
-
-# rapid validation errors indicate scans or attacks in progress
-# org.owasp.esapi.errors.ValidationException.count=10
-# org.owasp.esapi.errors.ValidationException.interval=10
-# org.owasp.esapi.errors.ValidationException.actions=log,logout
-
-# sessions jumping between hosts indicates session hijacking
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.count=2
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.interval=10
-IntrusionDetector.org.owasp.esapi.errors.AuthenticationHostException.actions=log,logout
-
-
-#===========================================================================
-# ESAPI Validation
-#
-# The ESAPI Validator works on regular expressions with defined names. You can define names
-# either here, or you may define application specific patterns in a separate file defined below.
-# This allows enterprises to specify both organizational standards as well as application specific
-# validation rules.
-#
-Validator.ConfigurationFile=validation.properties
-
-# Validators used by ESAPI
-Validator.AccountName=^[a-zA-Z0-9]{3,20}$
-Validator.SystemCommand=^[a-zA-Z\\-\\/]{1,64}$
-Validator.RoleName=^[a-z]{1,20}$
-
-#the word TEST below should be changed to your application
-#name - only relative URL's are supported
-Validator.Redirect=^\\/test.*$
-
-# Global HTTP Validation Rules
-# Values with Base64 encoded data (e.g. encrypted state) will need at least [a-zA-Z0-9\/+=]
-Validator.HTTPScheme=^(http|https)$
-Validator.HTTPServerName=^[a-zA-Z0-9_.\\-]*$
-Validator.HTTPParameterName=^[a-zA-Z0-9_]{1,32}$
-Validator.HTTPParameterValue=^[a-zA-Z0-9.\\-\\/+=_ ]*$
-Validator.HTTPCookieName=^[a-zA-Z0-9\\-_]{1,32}$
-Validator.HTTPCookieValue=^[a-zA-Z0-9\\-\\/+=_ ]*$
-Validator.HTTPHeaderName=^[a-zA-Z0-9\\-_]{1,32}$
-Validator.HTTPHeaderValue=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
-Validator.HTTPContextPath=^[a-zA-Z0-9.\\-\\/_]*$
-Validator.HTTPServletPath=^[a-zA-Z0-9.\\-\\/_]*$
-Validator.HTTPPath=^[a-zA-Z0-9.\\-_]*$
-Validator.HTTPQueryString=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ %]*$
-Validator.HTTPURI=^[a-zA-Z0-9()\\-=\\*\\.\\?;,+\\/:&_ ]*$
-Validator.HTTPURL=^.*$
-Validator.HTTPJSESSIONID=^[A-Z0-9]{10,30}$
-
-# Validation of file related input
-Validator.FileName=^[a-zA-Z0-9!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$
-Validator.DirectoryName=^[a-zA-Z0-9:/\\\\!@#$%^&{}\\[\\]()_+\\-=,.~'` ]{1,255}$

http://git-wip-us.apache.org/repos/asf/hbase/blob/d82ae421/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index af49452..87865d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -828,7 +828,6 @@
               <exclude>**/patchprocess/**</exclude>
               <exclude>src/main/site/resources/repo/**</exclude>
               <exclude>**/dependency-reduced-pom.xml</exclude>
-              <exclude>**/ESAPI.properties</exclude>
               <exclude>**/rat.txt</exclude>
             </excludes>
           </configuration>


[04/32] hbase git commit: HBASE-15202 Reduce garbage while setting response (Ram)

Posted by sy...@apache.org.
HBASE-15202 Reduce garbage while setting response (Ram)


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

Branch: refs/heads/hbase-12439
Commit: 7b33a740b10b05b50f8e9d3b2a1ef37593cb6eb3
Parents: f5fba2b
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Feb 4 23:23:31 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Feb 4 23:23:31 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 51 +++++++++++++++++---
 1 file changed, 43 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7b33a740/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 a9c64a3..98669e9 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
@@ -131,6 +131,7 @@ import org.codehaus.jackson.map.ObjectMapper;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
 import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
 import com.google.protobuf.ServiceException;
@@ -427,14 +428,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
         Message header = headerBuilder.build();
 
-        // Organize the response as a set of bytebuffers rather than collect it all together inside
-        // one big byte array; save on allocations.
-        ByteBuffer bbHeader = IPCUtil.getDelimitedMessageAsByteBuffer(header);
-        ByteBuffer bbResult = IPCUtil.getDelimitedMessageAsByteBuffer(result);
-        int totalSize = bbHeader.capacity() + (bbResult == null? 0: bbResult.limit()) +
-          (this.cellBlock == null? 0: this.cellBlock.limit());
-        ByteBuffer bbTotalSize = ByteBuffer.wrap(Bytes.toBytes(totalSize));
-        bc = new BufferChain(bbTotalSize, bbHeader, bbResult, this.cellBlock);
+        byte[] b = createHeaderAndMessageBytes(result, header);
+
+        bc = new BufferChain(ByteBuffer.wrap(b), this.cellBlock);
+
         if (connection.useWrap) {
           bc = wrapWithSasl(bc);
         }
@@ -454,6 +451,44 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       }
     }
 
+    private byte[] createHeaderAndMessageBytes(Message result, Message header)
+        throws IOException {
+      // Organize the response as a set of bytebuffers rather than collect it all together inside
+      // one big byte array; save on allocations.
+      int headerSerializedSize = 0, resultSerializedSize = 0, headerVintSize = 0,
+          resultVintSize = 0;
+      if (header != null) {
+        headerSerializedSize = header.getSerializedSize();
+        headerVintSize = CodedOutputStream.computeRawVarint32Size(headerSerializedSize);
+      }
+      if (result != null) {
+        resultSerializedSize = result.getSerializedSize();
+        resultVintSize = CodedOutputStream.computeRawVarint32Size(resultSerializedSize);
+      }
+      // calculate the total size
+      int totalSize = headerSerializedSize + headerVintSize
+          + (resultSerializedSize + resultVintSize)
+          + (this.cellBlock == null ? 0 : this.cellBlock.limit());
+      // The byte[] should also hold the totalSize of the header, message and the cellblock
+      byte[] b = new byte[headerSerializedSize + headerVintSize + resultSerializedSize
+          + resultVintSize + Bytes.SIZEOF_INT];
+      // The RpcClient expects the int to be in a format that code be decoded by
+      // the DataInputStream#readInt(). Hence going with the Bytes.toBytes(int)
+      // form of writing int.
+      Bytes.putInt(b, 0, totalSize);
+      CodedOutputStream cos = CodedOutputStream.newInstance(b, Bytes.SIZEOF_INT,
+          b.length - Bytes.SIZEOF_INT);
+      if (header != null) {
+        cos.writeMessageNoTag(header);
+      }
+      if (result != null) {
+        cos.writeMessageNoTag(result);
+      }
+      cos.flush();
+      cos.checkNoSpaceLeft();
+      return b;
+    }
+
     private BufferChain wrapWithSasl(BufferChain bc)
         throws IOException {
       if (!this.connection.useSasl) return bc;


[24/32] hbase git commit: HBASE-14770 RowCounter argument input parse error

Posted by sy...@apache.org.
HBASE-14770 RowCounter argument input parse error

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


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

Branch: refs/heads/hbase-12439
Commit: 59b03c77de659f426b6290e19faed76787b72872
Parents: efc7a0d
Author: Adrian Muraru <am...@adobe.com>
Authored: Sun Jan 24 18:10:35 2016 +0200
Committer: stack <st...@apache.org>
Committed: Fri Feb 5 21:41:30 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/mapreduce/RowCounter.java      | 17 ++++++------
 .../hadoop/hbase/mapreduce/TestRowCounter.java  | 27 ++++++++++++++++----
 2 files changed, 30 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59b03c77/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
index 145b366..8522a61 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
@@ -118,10 +118,7 @@ public class RowCounter extends Configured implements Tool {
         }
         startKey = startEnd[0];
         endKey = startEnd[1];
-      }
-      if (startTime < endTime) {
-        printUsage("--endtime=" + endTime + " needs to be greater than --starttime=" + startTime);
-        return null;
+        continue;
       }
       if (args[i].startsWith(startTimeArgKey)) {
         startTime = Long.parseLong(args[i].substring(startTimeArgKey.length()));
@@ -136,11 +133,13 @@ public class RowCounter extends Configured implements Tool {
             Long.parseLong(args[i].substring(expectedCountArg.length())));
         continue;
       }
-      else {
-        // if no switch, assume column names
-        sb.append(args[i]);
-        sb.append(" ");
-      }
+      // if no switch, assume column names
+      sb.append(args[i]);
+      sb.append(" ");
+    }
+    if (endTime < startTime) {
+      printUsage("--endtime=" + endTime + " needs to be greater than --starttime=" + startTime);
+      return null;
     }
 
     Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));

http://git-wip-us.apache.org/repos/asf/hbase/blob/59b03c77/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
index 8501164..6657d0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
@@ -57,6 +57,7 @@ public class TestRowCounter {
   private static final Log LOG = LogFactory.getLog(TestRowCounter.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final static String TABLE_NAME = "testRowCounter";
+  private final static String TABLE_NAME_TS_RANGE = "testRowCounter_ts_range";
   private final static String COL_FAM = "col_fam";
   private final static String COL1 = "c1";
   private final static String COL2 = "c2";
@@ -138,6 +139,21 @@ public class TestRowCounter {
     runRowCount(args, 10);
   }
 
+
+  /**
+   * Test a case when the column specified in command line arguments is
+   * exclusive for few rows and also a row range filter is specified
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testRowCounterColumnAndRowRange() throws Exception {
+    String[] args = new String[] {
+            TABLE_NAME, "--range=rov,rox", COL_FAM + ":" + COL1
+    };
+    runRowCount(args, 8);
+  }
+
    /**
    * Test a case when the timerange is specified with --starttime and --endtime options
    *
@@ -154,7 +170,8 @@ public class TestRowCounter {
     long ts;
 
     // clean up content of TABLE_NAME
-    Table table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
+    Table table = TEST_UTIL.createTable(TableName.valueOf(TABLE_NAME_TS_RANGE), Bytes.toBytes(COL_FAM));
+
     ts = System.currentTimeMillis();
     put1.addColumn(family, col1, ts, Bytes.toBytes("val1"));
     table.put(put1);
@@ -168,28 +185,28 @@ public class TestRowCounter {
     table.close();
 
     String[] args = new String[] {
-        TABLE_NAME, COL_FAM + ":" + COL1,
+        TABLE_NAME_TS_RANGE, COL_FAM + ":" + COL1,
         "--starttime=" + 0,
         "--endtime=" + ts
     };
     runRowCount(args, 1);
 
     args = new String[] {
-        TABLE_NAME, COL_FAM + ":" + COL1,
+        TABLE_NAME_TS_RANGE, COL_FAM + ":" + COL1,
         "--starttime=" + 0,
         "--endtime=" + (ts - 10)
     };
     runRowCount(args, 1);
 
     args = new String[] {
-        TABLE_NAME, COL_FAM + ":" + COL1,
+        TABLE_NAME_TS_RANGE, COL_FAM + ":" + COL1,
         "--starttime=" + ts,
         "--endtime=" + (ts + 1000)
     };
     runRowCount(args, 2);
 
     args = new String[] {
-        TABLE_NAME, COL_FAM + ":" + COL1,
+        TABLE_NAME_TS_RANGE, COL_FAM + ":" + COL1,
         "--starttime=" + (ts - 30 * 1000),
         "--endtime=" + (ts + 30 * 1000),
     };