You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2016/06/07 23:06:49 UTC

[1/8] hbase git commit: HBASE-13532 Make UnknownScannerException less scary by giving more information in the exception string. (Apekshit)

Repository: hbase
Updated Branches:
  refs/heads/0.98 8e132aa09 -> 95694adc5


HBASE-13532 Make UnknownScannerException less scary by giving more information in the exception string. (Apekshit)

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: fb1995f58644ff55e6ea4f3ebb56bf807cad480b
Parents: 8e132aa
Author: Apekshit <ap...@gmail.com>
Authored: Tue May 10 19:11:51 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 10:53:56 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/HRegionServer.java | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fb1995f5/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 072ca2b..a66776c 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
@@ -3220,9 +3220,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
       if (request.hasScannerId()) {
         rsh = scanners.get(scannerName);
         if (rsh == null) {
-          LOG.info("Client tried to access missing scanner " + scannerName);
+          LOG.warn("Client tried to access missing scanner " + scannerName);
           throw new UnknownScannerException(
-            "Name: " + scannerName + ", already closed?");
+            "Unknown scanner '" + scannerName + "'. This can happen due to any of the following "
+                + "reasons: a) Scanner id given is wrong, b) Scanner lease expired because of "
+                + "long wait between consecutive client checkins, c) Server may be closing down, "
+                + "d) RegionServer restart during upgrade.\nIf the issue is due to reason (b), a "
+                + "possible fix would be increasing the value of"
+                + "'hbase.client.scanner.timeout.period' configuration.");
         }
         scanner = rsh.s;
         HRegionInfo hri = scanner.getRegionInfo();


[3/8] hbase git commit: HBASE-15769 Perform validation on cluster key for add_peer (Matt Warhaftig)

Posted by ap...@apache.org.
HBASE-15769 Perform validation on cluster key for add_peer (Matt Warhaftig)

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 458378d7089a18b325a35c4c99ca15272fb033b4
Parents: 2d3ef97
Author: tedyu <yu...@gmail.com>
Authored: Tue May 17 13:26:45 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 14:29:22 2016 -0700

----------------------------------------------------------------------
 .../replication/ReplicationPeersZKImpl.java     | 13 ++++++--
 .../apache/hadoop/hbase/zookeeper/ZKConfig.java | 11 +++++++
 .../hadoop/hbase/zookeeper/TestZKConfig.java    | 20 ++++++------
 .../replication/TestReplicationStateBasic.java  | 34 +++++++++++++++++++-
 .../src/main/ruby/shell/commands/add_peer.rb    |  7 ++--
 5 files changed, 67 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/458378d7/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 632894c..0179574 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -107,10 +108,18 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
             + " because that id already exists.");
       }
       
-      if(id.contains("-")){
+      if (id.contains("-")) {
         throw new IllegalArgumentException("Found invalid peer name:" + id);
       }
-      
+
+      if (peerConfig.getClusterKey() != null) {
+        try {
+          ZKConfig.validateClusterKey(peerConfig.getClusterKey());
+        } catch (IOException ioe) {
+          throw new IllegalArgumentException(ioe.getMessage());
+        }
+      }
+
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
       List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
       ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),

http://git-wip-us.apache.org/repos/asf/hbase/blob/458378d7/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
index d23a607..af6b98f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java
@@ -339,12 +339,23 @@ public final class ZKConfig {
     String[] parts = key.split(":");
 
     if (parts.length == 3) {
+      if (!parts[2].matches("/.*[^/]")) {
+        throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
+            HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
+            + HConstants.ZOOKEEPER_ZNODE_PARENT);
+      }
       return new ZKClusterKey(parts [0], Integer.parseInt(parts [1]), parts [2]);
     }
 
     if (parts.length > 3) {
       // The quorum could contain client port in server:clientport format, try to transform more.
       String zNodeParent = parts [parts.length - 1];
+      if (!zNodeParent.matches("/.*[^/]")) {
+        throw new IOException("Cluster key passed " + key + " is invalid, the format should be:"
+            + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
+            + HConstants.ZOOKEEPER_ZNODE_PARENT);
+      }
+
       String clientPort = parts [parts.length - 2];
 
       // The first part length is the total length minus the lengths of other parts and minus 2 ":"

http://git-wip-us.apache.org/repos/asf/hbase/blob/458378d7/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
index 7879aea..945b291 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
@@ -60,10 +60,10 @@ public class TestZKConfig {
 
   @Test
   public void testClusterKey() throws Exception {
-    testKey("server", 2181, "hbase");
-    testKey("server1,server2,server3", 2181, "hbase");
+    testKey("server", 2181, "/hbase");
+    testKey("server1,server2,server3", 2181, "/hbase");
     try {
-      ZKConfig.validateClusterKey("2181:hbase");
+      ZKConfig.validateClusterKey("2181:/hbase");
     } catch (IOException ex) {
       // OK
     }
@@ -72,19 +72,19 @@ public class TestZKConfig {
   @Test
   public void testClusterKeyWithMultiplePorts() throws Exception {
     // server has different port than the default port
-    testKey("server1:2182", 2181, "hbase", true);
+    testKey("server1:2182", 2181, "/hbase", true);
     // multiple servers have their own port
-    testKey("server1:2182,server2:2183,server3:2184", 2181, "hbase", true);
+    testKey("server1:2182,server2:2183,server3:2184", 2181, "/hbase", true);
     // one server has no specified port, should use default port
-    testKey("server1:2182,server2,server3:2184", 2181, "hbase", true);
+    testKey("server1:2182,server2,server3:2184", 2181, "/hbase", true);
     // the last server has no specified port, should use default port
-    testKey("server1:2182,server2:2183,server3", 2181, "hbase", true);
+    testKey("server1:2182,server2:2183,server3", 2181, "/hbase", true);
     // multiple servers have no specified port, should use default port for those servers
-    testKey("server1:2182,server2,server3:2184,server4", 2181, "hbase", true);
+    testKey("server1:2182,server2,server3:2184,server4", 2181, "/hbase", true);
     // same server, different ports
-    testKey("server1:2182,server1:2183,server1", 2181, "hbase", true);
+    testKey("server1:2182,server1:2183,server1", 2181, "/hbase", true);
     // mix of same server/different port and different server
-    testKey("server1:2182,server2:2183,server1", 2181, "hbase", true);
+    testKey("server1:2182,server2:2183,server1", 2181, "/hbase", true);
   }
 
   private void testKey(String ensemble, int port, String znode)

http://git-wip-us.apache.org/repos/asf/hbase/blob/458378d7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 696c130..da04f41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -160,6 +160,38 @@ public abstract class TestReplicationStateBasic {
   }
 
   @Test
+  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
+    rp.init();
+
+    try {
+      rp.addPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"), null);
+      fail("Should throw an IllegalArgumentException because "
+            + "zookeeper.znode.parent is missing leading '/'.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.addPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"), null);
+      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.addPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"), null);
+      fail("Should throw an IllegalArgumentException because "
+          + "hbase.zookeeper.property.clientPort is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+  }
+
+
+  @Test
   public void testReplicationPeers() throws Exception {
     rp.init();
 
@@ -268,7 +300,7 @@ public abstract class TestReplicationStateBasic {
         rq3.addLog("qId" + i, "filename" + j);
       }
       //Add peers for the corresponding queues so they are not orphans
-      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("bogus" + i), null);
+      rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i), null);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/458378d7/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
index be01041..0fcdd3d 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
@@ -31,11 +31,8 @@ This gives a full path for HBase to connect to another HBase cluster. An optiona
 table column families identifies which column families will be replicated to the peer cluster.
 Examples:
 
-  hbase> add_peer '1', "server1.cie.com:2181:/hbase"
-  hbase> add_peer '2', "zk1,zk2,zk3:2182:/hbase-prod"
-  hbase> add_peer '3', "zk4,zk5,zk6:11000:/hbase-test", "table1; table2:cf1; table3:cf1,cf2"
-  hbase> add_peer '4', CLUSTER_KEY => "server1.cie.com:2181:/hbase"
-  hbase> add_peer '5', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase"
+  hbase> add_peer '2', CLUSTER_KEY => "zk1,zk2,zk3:2182:/hbase-prod",
     TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
 
 For a custom replication endpoint, the ENDPOINT_CLASSNAME can be provided. Two optional arguments


[6/8] hbase git commit: HBASE-15727 Canary Tool for Zookeeper (churro morales)

Posted by ap...@apache.org.
HBASE-15727 Canary Tool for Zookeeper (churro morales)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 60958f591258fabca627ec087286112793fe6dbf
Parents: fee078d
Author: tedyu <yu...@gmail.com>
Authored: Thu Jun 2 10:17:42 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 15:10:58 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/tool/Canary.java    | 145 ++++++++++++++++++-
 .../hadoop/hbase/tool/TestCanaryTool.java       |  19 +++
 2 files changed, 162 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/60958f59/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index cfb9860..4f2e5fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -19,8 +19,14 @@
 
 package org.apache.hadoop.hbase.tool;
 
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT;
+import static org.apache.hadoop.hbase.HConstants.ZOOKEEPER_ZNODE_PARENT;
+
+import com.google.common.collect.Lists;
+
 import java.io.Closeable;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -74,9 +80,15 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.client.ConnectStringParser;
+import org.apache.zookeeper.data.Stat;
 
 import com.google.protobuf.ServiceException;
 
@@ -84,12 +96,15 @@ import com.google.protobuf.ServiceException;
  * HBase Canary Tool, that that can be used to do
  * "canary monitoring" of a running HBase cluster.
  *
- * Here are two modes
+ * Here are three modes
  * 1. region mode - Foreach region tries to get one row per column family
  * and outputs some information about failure or latency.
  *
  * 2. regionserver mode - Foreach regionserver tries to get one row from one table
  * selected randomly and outputs some information about failure or latency.
+ *
+ * 3. zookeeper mode - for each zookeeper instance, selects a zNode and
+ * outputs some information about failure or latency.
  */
 public final class Canary implements Tool {
   // Sink interface used by the canary to outputs information
@@ -186,6 +201,57 @@ public final class Canary implements Tool {
     }
   }
 
+  public static class ZookeeperStdOutSink extends StdOutSink implements ExtendedSink {
+    @Override public void publishReadFailure(String zNode, String server) {
+      incReadFailureCount();
+      LOG.error(String.format("Read from zNode:%s on zookeeper instance:%s", zNode, server));
+    }
+
+    @Override public void publishReadTiming(String znode, String server, long msTime) {
+      LOG.info(String.format("Read from zNode:%s on zookeeper instance:%s in %dms",
+          znode, server, msTime));
+    }
+  }
+
+  static class ZookeeperTask implements Callable<Void> {
+    private final HConnection connection;
+    private final String host;
+    private String znode;
+    private final int timeout;
+    private ZookeeperStdOutSink sink;
+
+    public ZookeeperTask(HConnection connection, String host, String znode, int timeout,
+        ZookeeperStdOutSink sink) {
+      this.connection = connection;
+      this.host = host;
+      this.znode = znode;
+      this.timeout = timeout;
+      this.sink = sink;
+    }
+
+    @Override public Void call() throws Exception {
+      ZooKeeper zooKeeper = null;
+      try {
+        zooKeeper = new ZooKeeper(host, timeout, EmptyWatcher.instance);
+        Stat exists = zooKeeper.exists(znode, false);
+        StopWatch stopwatch = new StopWatch();
+        stopwatch.start();
+        zooKeeper.getData(znode, false, exists);
+        stopwatch.stop();
+        sink.publishReadTiming(znode, host, stopwatch.getTime());
+      } catch (KeeperException e) {
+        sink.publishReadFailure(znode, host);
+      } catch (InterruptedException e) {
+        sink.publishReadFailure(znode, host);
+      } finally {
+        if (zooKeeper != null) {
+          zooKeeper.close();
+        }
+      }
+      return null;
+    }
+  }
+
   /**
    * For each column family of the region tries to get one row and outputs the latency, or the
    * failure.
@@ -455,6 +521,8 @@ public final class Canary implements Tool {
   private long timeout = DEFAULT_TIMEOUT;
   private boolean failOnError = true;
   private boolean regionServerMode = false;
+  private boolean zookeeperMode = false;
+
   private boolean writeSniffing = false;
   private boolean treatFailureAsError = false;
   private TableName writeTableName = DEFAULT_WRITE_TABLE_NAME;
@@ -514,6 +582,8 @@ public final class Canary implements Tool {
             System.err.println("-interval needs a numeric value argument.");
             printUsageAndExit();
           }
+        } else if (cmd.equals("-zookeeper")) {
+          this.zookeeperMode = true;
         } else if(cmd.equals("-regionserver")) {
           this.regionServerMode = true;
         } else if(cmd.equals("-writeSniffing")) {
@@ -564,6 +634,15 @@ public final class Canary implements Tool {
         index = i;
       }
     }
+
+    if (this.zookeeperMode) {
+      if (this.regionServerMode || this.writeSniffing) {
+        System.err.println("-zookeeper is exclusive and cannot be combined with "
+            + "other modes.");
+        printUsageAndExit();
+      }
+    }
+
     return index;
   }
 
@@ -641,6 +720,8 @@ public final class Canary implements Tool {
     System.err.println("   -help          Show this help and exit.");
     System.err.println("   -regionserver  replace the table argument to regionserver,");
     System.err.println("      which means to enable regionserver mode");
+    System.err.println("   -zookeeper    Tries to grab zookeeper.znode.parent ");
+    System.err.println("      on each zookeeper instance");
     System.err.println("   -daemon        Continuous check at defined intervals.");
     System.err.println("   -interval <N>  Interval between checks (sec)");
     System.err.println("   -e             Use table/regionserver as regular expression");
@@ -678,6 +759,10 @@ public final class Canary implements Tool {
       monitor =
           new RegionServerMonitor(connection, monitorTargets, this.useRegExp,
               (ExtendedSink) this.sink, this.executor, this.treatFailureAsError);
+    } else if (this.zookeeperMode) {
+      monitor =
+          new ZookeeperMonitor(connection, monitorTargets, this.useRegExp,
+              (ZookeeperStdOutSink) this.sink, this.executor, this.treatFailureAsError);
     } else {
       monitor =
           new RegionMonitor(connection, monitorTargets, this.useRegExp, this.sink, this.executor,
@@ -1060,6 +1145,62 @@ public final class Canary implements Tool {
       }
     }
   }
+
+  //  monitor for zookeeper mode
+  private static class ZookeeperMonitor extends Monitor {
+    private List<String> hosts;
+    private final String znode;
+    private final int timeout;
+
+    protected ZookeeperMonitor(HConnection connection, String[] monitorTargets, boolean useRegExp,
+        ExtendedSink sink, ExecutorService executor, boolean treatFailureAsError)  {
+      super(connection, monitorTargets, useRegExp, sink, executor, treatFailureAsError);
+      Configuration configuration = connection.getConfiguration();
+      znode =
+          configuration.get(ZOOKEEPER_ZNODE_PARENT,
+              DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+      timeout = configuration
+          .getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
+      ConnectStringParser parser =
+          new ConnectStringParser(ZKConfig.getZKQuorumServersString(configuration));
+      hosts = Lists.newArrayList();
+      for (InetSocketAddress server : parser.getServerAddresses()) {
+        hosts.add(server.toString());
+      }
+    }
+
+    @Override public void run() {
+      List<ZookeeperTask> tasks = Lists.newArrayList();
+      for (final String host : hosts) {
+        tasks.add(new ZookeeperTask(connection, host, znode, timeout, getSink()));
+      }
+      try {
+        for (Future<Void> future : this.executor.invokeAll(tasks)) {
+          try {
+            future.get();
+          } catch (ExecutionException e) {
+            LOG.error("Sniff zookeeper failed!", e);
+            this.errorCode = ERROR_EXIT_CODE;
+          }
+        }
+      } catch (InterruptedException e) {
+        this.errorCode = ERROR_EXIT_CODE;
+        Thread.currentThread().interrupt();
+        LOG.error("Sniff zookeeper interrupted!", e);
+      }
+      this.done = true;
+    }
+
+
+    private ZookeeperStdOutSink getSink() {
+      if (!(sink instanceof ZookeeperStdOutSink)) {
+        throw new RuntimeException("Can only write to zookeeper sink");
+      }
+      return ((ZookeeperStdOutSink) sink);
+    }
+  }
+
+
   // a monitor for regionserver mode
   private static class RegionServerMonitor extends Monitor {
 
@@ -1249,7 +1390,7 @@ public final class Canary implements Tool {
     AuthUtil.launchAuthChore(conf);  
 
     int numThreads = conf.getInt("hbase.canary.threads.num", MAX_THREADS_NUM);
-    LOG.info("Number of exection threads " + numThreads);
+    LOG.info("Number of execution threads " + numThreads);
 
     ExecutorService executor = new ScheduledThreadPoolExecutor(numThreads);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/60958f59/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index b765177..18c269a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Appender;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.spi.LoggingEvent;
+import org.apache.hadoop.hbase.HConstants;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -44,6 +45,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
 import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.spy;
@@ -76,6 +78,23 @@ public class TestCanaryTool {
   Appender mockAppender;
 
   @Test
+  public void testBasicZookeeperCanaryWorks() throws Exception {
+    int port = testingUtility.getZkCluster().getClientPort();
+    testingUtility.getConfiguration().set(HConstants.ZOOKEEPER_QUORUM,
+        "localhost:" + port + "/hbase");
+    ExecutorService executor = new ScheduledThreadPoolExecutor(2);
+    Canary.ZookeeperStdOutSink sink = spy(new Canary.ZookeeperStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String[] args = { "-t", "10000", "-zookeeper" };
+    ToolRunner.run(testingUtility.getConfiguration(), canary, args);
+
+    String baseZnode = testingUtility.getConfiguration()
+        .get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+    verify(sink, atLeastOnce())
+        .publishReadTiming(eq(baseZnode), eq("localhost:" + port), anyLong());
+  }
+
+  @Test
   public void testBasicCanaryWorks() throws Exception {
     TableName tableName = TableName.valueOf("testTable");
     HTable table = testingUtility.createTable(tableName, new byte[][] { FAMILY });


[7/8] hbase git commit: HBASE-15889. String case conversions are locale-sensitive, used without locale

Posted by ap...@apache.org.
HBASE-15889. String case conversions are locale-sensitive, used without locale

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

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
	hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
	hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/TimedOutTestsListener.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: e26800c027fc010fd72c4bb2b78997ea30466606
Parents: 60958f5
Author: Sean Mackrory <ma...@apache.org>
Authored: Wed May 25 07:49:01 2016 -0600
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 15:35:12 2016 -0700

----------------------------------------------------------------------
 .../classification/tools/StabilityOptions.java   |  5 +++--
 .../apache/hadoop/hbase/HColumnDescriptor.java   | 19 ++++++++++---------
 .../java/org/apache/hadoop/hbase/ServerName.java |  3 ++-
 .../hadoop/hbase/catalog/CatalogTracker.java     |  3 ++-
 .../hadoop/hbase/filter/SubstringComparator.java |  7 ++++---
 .../org/apache/hadoop/hbase/ipc/RpcClient.java   |  7 ++++---
 .../apache/hadoop/hbase/security/SaslUtil.java   |  7 ++++---
 .../org/apache/hadoop/hbase/util/PoolMap.java    |  3 ++-
 .../hbase/io/crypto/KeyStoreKeyProvider.java     |  3 ++-
 .../ipc/MetricsHBaseServerSourceFactoryImpl.java |  3 ++-
 .../hbase/test/MetricsAssertHelperImpl.java      |  3 ++-
 .../ipc/MetricsHBaseServerSourceFactoryImpl.java |  3 ++-
 .../hbase/test/MetricsAssertHelperImpl.java      |  3 ++-
 .../apache/hadoop/hbase/HBaseClusterManager.java |  7 ++++---
 .../hadoop/hbase/RESTApiClusterManager.java      |  9 +++++----
 .../StripeCompactionsPerformanceEvaluation.java  |  3 ++-
 .../hadoop/hbase/rest/filter/GzipFilter.java     |  5 +++--
 .../apache/hadoop/hbase/mapreduce/Import.java    |  3 ++-
 .../hbase/regionserver/BaseRowProcessor.java     |  3 ++-
 .../hadoop/hbase/regionserver/HRegion.java       |  3 ++-
 .../hbase/security/HBaseSaslRpcServer.java       |  3 ++-
 .../hadoop/hbase/snapshot/CreateSnapshot.java    |  3 ++-
 .../hadoop/hbase/util/CompressionTest.java       |  5 +++--
 .../hadoop/hbase/util/DirectMemoryUtils.java     |  3 ++-
 .../org/apache/hadoop/hbase/util/FSUtils.java    |  3 ++-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java  |  3 ++-
 .../hadoop/hbase/util/ServerCommandLine.java     |  5 +++--
 .../hadoop/hbase/PerformanceEvaluation.java      |  5 +++--
 .../hbase/TestIPv6NIOServerSocketChannel.java    |  3 ++-
 .../mapreduce/MultiTableInputFormatTestBase.java |  5 +++--
 .../mapreduce/TestLoadIncrementalHFiles.java     |  3 ++-
 .../mapreduce/TestTableInputFormatScanBase.java  |  9 +++++----
 .../regionserver/DataBlockEncodingTool.java      |  9 +++++----
 .../apache/hadoop/hbase/util/LoadTestTool.java   |  3 ++-
 .../thrift/HThreadedSelectorServerArgs.java      |  3 ++-
 .../hadoop/hbase/thrift/ThriftUtilities.java     |  3 ++-
 36 files changed, 103 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
----------------------------------------------------------------------
diff --git a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
index f3169ba..217ad12 100644
--- a/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
+++ b/hbase-annotations/src/main/java/org/apache/hadoop/hbase/classification/tools/StabilityOptions.java
@@ -21,6 +21,7 @@ import com.sun.javadoc.DocErrorReporter;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 
 class StabilityOptions {
   public static final String STABLE_OPTION = "-stable";
@@ -28,7 +29,7 @@ class StabilityOptions {
   public static final String UNSTABLE_OPTION = "-unstable";
 
   public static Integer optionLength(String option) {
-    String opt = option.toLowerCase();
+    String opt = option.toLowerCase(Locale.ROOT);
     if (opt.equals(UNSTABLE_OPTION)) return 1;
     if (opt.equals(EVOLVING_OPTION)) return 1;
     if (opt.equals(STABLE_OPTION)) return 1;
@@ -38,7 +39,7 @@ class StabilityOptions {
   public static void validOptions(String[][] options,
       DocErrorReporter reporter) {
     for (int i = 0; i < options.length; i++) {
-      String opt = options[i][0].toLowerCase();
+      String opt = options[i][0].toLowerCase(Locale.ROOT);
       if (opt.equals(UNSTABLE_OPTION)) {
 	RootDocProcessor.stability = UNSTABLE_OPTION;
       } else if (opt.equals(EVOLVING_OPTION)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 15aee9d..d83487c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
@@ -445,11 +446,11 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
     setBlockCacheEnabled(blockCacheEnabled);
     setTimeToLive(timeToLive);
     setCompressionType(Compression.Algorithm.
-      valueOf(compression.toUpperCase()));
+      valueOf(compression.toUpperCase(Locale.ROOT)));
     setDataBlockEncoding(DataBlockEncoding.
-        valueOf(dataBlockEncoding.toUpperCase()));
+        valueOf(dataBlockEncoding.toUpperCase(Locale.ROOT)));
     setBloomFilterType(BloomType.
-      valueOf(bloomFilter.toUpperCase()));
+      valueOf(bloomFilter.toUpperCase(Locale.ROOT)));
     setBlocksize(blocksize);
     setScope(scope);
   }
@@ -568,7 +569,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
     if (n == null) {
       return Compression.Algorithm.NONE;
     }
-    return Compression.Algorithm.valueOf(n.toUpperCase());
+    return Compression.Algorithm.valueOf(n.toUpperCase(Locale.ROOT));
   }
 
   /** @return compression type being used for the column family for major
@@ -578,7 +579,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
     if (n == null) {
       return getCompression();
     }
-    return Compression.Algorithm.valueOf(n.toUpperCase());
+    return Compression.Algorithm.valueOf(n.toUpperCase(Locale.ROOT));
   }
 
   /** @return maximum number of versions */
@@ -673,7 +674,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
    * @return this (for chained invocation)
    */
   public HColumnDescriptor setCompressionType(Compression.Algorithm type) {
-    return setValue(COMPRESSION, type.getName().toUpperCase());
+    return setValue(COMPRESSION, type.getName().toUpperCase(Locale.ROOT));
   }
 
   /** @return data block encoding algorithm used on disk */
@@ -760,7 +761,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
    */
   public HColumnDescriptor setCompactionCompressionType(
       Compression.Algorithm type) {
-    return setValue(COMPRESSION_COMPACT, type.getName().toUpperCase());
+    return setValue(COMPRESSION_COMPACT, type.getName().toUpperCase(Locale.ROOT));
   }
 
   /**
@@ -786,7 +787,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
     String value = getValue(KEEP_DELETED_CELLS);
     if (value != null) {
       // toUpperCase for backwards compatibility
-      return KeepDeletedCells.valueOf(value.toUpperCase());
+      return KeepDeletedCells.valueOf(value.toUpperCase(Locale.ROOT));
     }
     return DEFAULT_KEEP_DELETED;
   }
@@ -871,7 +872,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
     if (n == null) {
       n = DEFAULT_BLOOMFILTER;
     }
-    return BloomType.valueOf(n.toUpperCase());
+    return BloomType.valueOf(n.toUpperCase(Locale.ROOT));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
index 36a67fe..ec8fcb2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.regex.Pattern;
 
 /**
@@ -229,7 +230,7 @@ public class ServerName implements Comparable<ServerName>, Serializable {
    */
   static String getServerName(String hostName, int port, long startcode) {
     final StringBuilder name = new StringBuilder(hostName.length() + 1 + 5 + 1 + 13);
-    name.append(hostName.toLowerCase());
+    name.append(hostName.toLowerCase(Locale.ROOT));
     name.append(SERVERNAME_SEPARATOR);
     name.append(port);
     name.append(SERVERNAME_SEPARATOR);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
index 08b7ae7..4c9e20f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/CatalogTracker.java
@@ -50,6 +50,7 @@ import java.net.NoRouteToHostException;
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.UnknownHostException;
+import java.util.Locale;
 
 /**
  * Tracks the availability of the catalog tables
@@ -383,7 +384,7 @@ public class CatalogTracker {
       } else if (cause != null && cause instanceof EOFException) {
         // Catch. Other end disconnected us.
       } else if (cause != null && cause.getMessage() != null &&
-        cause.getMessage().toLowerCase().contains("connection reset")) {
+        cause.getMessage().toLowerCase(Locale.ROOT).contains("connection reset")) {
         // Catch. Connection reset.
       } else {
         throw ioe;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
index 5eb3703..1f0043c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.filter;
 
 import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.Locale;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -52,8 +53,8 @@ public class SubstringComparator extends ByteArrayComparable {
    * @param substr the substring
    */
   public SubstringComparator(String substr) {
-    super(Bytes.toBytes(substr.toLowerCase()));
-    this.substr = substr.toLowerCase();
+    super(Bytes.toBytes(substr.toLowerCase(Locale.ROOT)));
+    this.substr = substr.toLowerCase(Locale.ROOT);
   }
 
   @Override
@@ -63,7 +64,7 @@ public class SubstringComparator extends ByteArrayComparable {
 
   @Override
   public int compareTo(byte[] value, int offset, int length) {
-    return Bytes.toString(value, offset, length).toLowerCase().contains(substr) ? 0
+    return Bytes.toString(value, offset, length).toLowerCase(Locale.ROOT).contains(substr) ? 0
         : 1;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index a06faf9..edeb328 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -39,6 +39,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
@@ -411,8 +412,8 @@ public class RpcClient {
           throw new IOException(
               "Can't obtain server Kerberos config key from SecurityInfo");
         }
-        serverPrincipal = SecurityUtil.getServerPrincipal(
-            conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
+        serverPrincipal = SecurityUtil.getServerPrincipal(conf.get(serverKey),
+          server.getAddress().getCanonicalHostName().toLowerCase(Locale.ROOT));
         if (LOG.isDebugEnabled()) {
           LOG.debug("RPC Server Kerberos principal name for service="
               + remoteId.getServiceName() + " is " + serverPrincipal);
@@ -770,7 +771,7 @@ public class RpcClient {
         final OutputStream out2) throws IOException {
       saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed,
           conf.get("hbase.rpc.protection", 
-              QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
+              QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
       return saslRpcClient.saslConnect(in2, out2);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
index 1c7a77d..9a18cf9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/SaslUtil.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
+import java.util.Locale;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -76,13 +77,13 @@ public class SaslUtil {
    */
   public static QualityOfProtection getQop(String stringQop) {
     QualityOfProtection qop = null;
-    if (QualityOfProtection.AUTHENTICATION.name().toLowerCase().equals(stringQop)
+    if (QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT).equals(stringQop)
         || QualityOfProtection.AUTHENTICATION.saslQop.equals(stringQop)) {
       qop = QualityOfProtection.AUTHENTICATION;
-    } else if (QualityOfProtection.INTEGRITY.name().toLowerCase().equals(stringQop)
+    } else if (QualityOfProtection.INTEGRITY.name().toLowerCase(Locale.ROOT).equals(stringQop)
         || QualityOfProtection.INTEGRITY.saslQop.equals(stringQop)) {
       qop = QualityOfProtection.INTEGRITY;
-    } else if (QualityOfProtection.PRIVACY.name().toLowerCase().equals(stringQop)
+    } else if (QualityOfProtection.PRIVACY.name().toLowerCase(Locale.ROOT).equals(stringQop)
         || QualityOfProtection.PRIVACY.saslQop.equals(stringQop)) {
       qop = QualityOfProtection.PRIVACY;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
index 7d27b7a..086cd1c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -255,7 +256,7 @@ public class PoolMap<K, V> implements Map<K, V> {
     }
 
     public static String fuzzyNormalize(String name) {
-      return name != null ? name.replaceAll("-", "").trim().toLowerCase() : "";
+      return name != null ? name.replaceAll("-", "").trim().toLowerCase(Locale.ROOT) : "";
     }
 
     public static PoolType fuzzyMatch(String name) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
index 24e03c0..1e35351 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/crypto/KeyStoreKeyProvider.java
@@ -30,6 +30,7 @@ import java.security.KeyStoreException;
 import java.security.NoSuchAlgorithmException;
 import java.security.UnrecoverableKeyException;
 import java.security.cert.CertificateException;
+import java.util.Locale;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -142,7 +143,7 @@ public class KeyStoreKeyProvider implements KeyProvider {
         throw new RuntimeException("KeyProvider scheme should specify KeyStore type");
       }
       // KeyStore expects instance type specifications in uppercase
-      store = KeyStore.getInstance(storeType.toUpperCase());
+      store = KeyStore.getInstance(storeType.toUpperCase(Locale.ROOT));
       processParameters(uri);
       load(uri);
     } catch (URISyntaxException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
index 96de011..90550dc 100644
--- a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
+++ b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.ipc;
 
 import java.util.HashMap;
+import java.util.Locale;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -48,7 +49,7 @@ public class MetricsHBaseServerSourceFactoryImpl extends MetricsHBaseServerSourc
       source = new MetricsHBaseServerSourceImpl(
           METRICS_NAME,
           METRICS_DESCRIPTION,
-          context.toLowerCase(),
+          context.toLowerCase(Locale.ROOT),
           context + METRICS_JMX_CONTEXT_SUFFIX, wrapper);
 
       //Store back in storage

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java b/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
index dd85ae0..4ed194f 100644
--- a/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
+++ b/hbase-hadoop1-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 
 import static org.junit.Assert.*;
@@ -226,6 +227,6 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
   }
 
   private String canonicalizeMetricName(String in) {
-    return in.toLowerCase().replaceAll("[^A-Za-z0-9 ]", "");
+    return in.toLowerCase(Locale.ROOT).replaceAll("[^A-Za-z0-9 ]", "");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
index 96de011..90550dc 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceFactoryImpl.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.ipc;
 
 import java.util.HashMap;
+import java.util.Locale;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -48,7 +49,7 @@ public class MetricsHBaseServerSourceFactoryImpl extends MetricsHBaseServerSourc
       source = new MetricsHBaseServerSourceImpl(
           METRICS_NAME,
           METRICS_DESCRIPTION,
-          context.toLowerCase(),
+          context.toLowerCase(Locale.ROOT),
           context + METRICS_JMX_CONTEXT_SUFFIX, wrapper);
 
       //Store back in storage

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
index 36e2696..85b2cee 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 
 import static org.junit.Assert.*;
@@ -247,6 +248,6 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
   }
 
   private String canonicalizeMetricName(String in) {
-    return in.toLowerCase().replaceAll("[^A-Za-z0-9 ]", "");
+    return in.toLowerCase(Locale.ROOT).replaceAll("[^A-Za-z0-9 ]", "");
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
index ba6a4a9..b0281fb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/HBaseClusterManager.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Locale;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
@@ -205,7 +206,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
     @Override
     public String getCommand(ServiceType service, Operation op) {
       return String.format("%s/bin/hbase-daemon.sh %s %s %s", hbaseHome, confDir,
-          op.toString().toLowerCase(), service);
+          op.toString().toLowerCase(Locale.ROOT), service);
     }
   }
 
@@ -235,7 +236,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
     @Override
     public String getCommand(ServiceType service, Operation op) {
       return String.format("%s/sbin/hadoop-daemon.sh %s %s %s", hadoopHome, confDir,
-          op.toString().toLowerCase(), service);
+          op.toString().toLowerCase(Locale.ROOT), service);
     }
   }
 
@@ -264,7 +265,7 @@ public class HBaseClusterManager extends Configured implements ClusterManager {
 
     @Override
     public String getCommand(ServiceType service, Operation op) {
-      return String.format("%s/bin/zkServer.sh %s", zookeeperHome, op.toString().toLowerCase());
+      return String.format("%s/bin/zkServer.sh %s", zookeeperHome, op.toString().toLowerCase(Locale.ROOT));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
index 6c5c45e..ce7d645 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/RESTApiClusterManager.java
@@ -37,6 +37,7 @@ import javax.xml.ws.http.HTTPException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 
 /**
@@ -273,8 +274,8 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
         if (role.get("hostRef").get("hostId").getTextValue().equals(hostId) &&
             role.get("type")
                 .getTextValue()
-                .toLowerCase()
-                .equals(roleType.toLowerCase())) {
+                .toLowerCase(Locale.ROOT)
+                .equals(roleType.toLowerCase(Locale.ROOT))) {
           roleValue = role.get(property).getTextValue();
           break;
         }
@@ -327,7 +328,7 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
     // APIs tend to take commands in lowercase, so convert them to save the trouble later.
     @Override
     public String toString() {
-      return name().toLowerCase();
+      return name().toLowerCase(Locale.ROOT);
     }
   }
 
@@ -347,4 +348,4 @@ public class RESTApiClusterManager extends Configured implements ClusterManager
   private enum Service {
     HBASE, HDFS, MAPREDUCE
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
index c48734c..7071c73 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
+import java.util.Locale;
 import java.util.Set;
 
 import org.apache.commons.cli.CommandLine;
@@ -111,7 +112,7 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
     } else {
       minValueSize = maxValueSize = Integer.parseInt(valueSize);
     }
-    String datagen = cmd.getOptionValue(DATAGEN_KEY, "default").toLowerCase();
+    String datagen = cmd.getOptionValue(DATAGEN_KEY, "default").toLowerCase(Locale.ROOT);
     if ("default".equals(datagen)) {
       dataGen = new MultiThreadedAction.DefaultDataGenerator(
           minValueSize, maxValueSize, 1, 1, new byte[][] { COLUMN_FAMILY });

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
index 4995b86..094ae0b 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GzipFilter.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.rest.filter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Set;
 import java.util.StringTokenizer;
 
@@ -65,11 +66,11 @@ public class GzipFilter implements Filter {
     String acceptEncoding = request.getHeader("accept-encoding");
     String contentType = request.getHeader("content-type");
     if ((contentEncoding != null) &&
-        (contentEncoding.toLowerCase().indexOf("gzip") > -1)) {
+        (contentEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) {
       request = new GZIPRequestWrapper(request);
     }
     if (((acceptEncoding != null) &&
-          (acceptEncoding.toLowerCase().indexOf("gzip") > -1)) ||
+          (acceptEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) ||
         ((contentType != null) && mimeTypes.contains(contentType))) {
       response = new GZIPResponseWrapper(response);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index 22a59a5..d14f339 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -28,6 +28,7 @@ import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
@@ -400,7 +401,7 @@ public class Import {
       filter = instantiateFilter(conf);
       String durabilityStr = conf.get(WAL_DURABILITY);
       if(durabilityStr != null){
-        durability = Durability.valueOf(durabilityStr.toUpperCase());
+        durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));
       }
       // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.
       ZooKeeperWatcher zkw = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
index 65375b8..be2bd91 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/BaseRowProcessor.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.UUID;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -61,7 +62,7 @@ implements RowProcessor<S,T> {
 
   @Override
   public String getName() {
-    return this.getClass().getSimpleName().toLowerCase();
+    return this.getClass().getSimpleName().toLowerCase(Locale.ROOT);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/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 c337c9c..90e9296 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
@@ -33,6 +33,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.NavigableSet;
@@ -6480,7 +6481,7 @@ public class HRegion implements HeapSize { // , Writable{
     }
     boolean majorCompact = false;
     if (args.length > 1) {
-      if (!args[1].toLowerCase().startsWith("major")) {
+      if (!args[1].toLowerCase(Locale.ROOT).startsWith("major")) {
         printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
       }
       majorCompact = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java
index 3ee7a9a..1b37efc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/HBaseSaslRpcServer.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.security;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
+import java.util.Locale;
 
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
@@ -50,7 +51,7 @@ public class HBaseSaslRpcServer {
 
   public static void init(Configuration conf) {
     SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection", 
-          QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
+          QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));
   }
 
   public static <T extends TokenIdentifier> T getIdentifier(String id,

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/CreateSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/CreateSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/CreateSnapshot.java
index e135143..75e8dea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/CreateSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/CreateSnapshot.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 
 import java.util.Arrays;
+import java.util.Locale;
 
 
 /**
@@ -63,7 +64,7 @@ public class CreateSnapshot extends AbstractHBaseTool {
             admin = new HBaseAdmin(conf);
             HBaseProtos.SnapshotDescription.Type type = HBaseProtos.SnapshotDescription.Type.FLUSH;
             if (snapshotType != null) {
-                type = HBaseProtos.SnapshotDescription.Type.valueOf(snapshotName.toUpperCase());
+                type = HBaseProtos.SnapshotDescription.Type.valueOf(snapshotName.toUpperCase(Locale.ROOT));
             }
 
             admin.snapshot(snapshotName, TableName.valueOf(tableName), type);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
index 355719d..2c6fadd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
+import java.util.Locale;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -47,7 +48,7 @@ public class CompressionTest {
   static final Log LOG = LogFactory.getLog(CompressionTest.class);
 
   public static boolean testCompression(String codec) {
-    codec = codec.toLowerCase();
+    codec = codec.toLowerCase(Locale.ROOT);
 
     Compression.Algorithm a;
 
@@ -103,7 +104,7 @@ public class CompressionTest {
 
     System.err.println(
       "Usage: CompressionTest <path> " +
-      StringUtils.join( Compression.Algorithm.values(), "|").toLowerCase() +
+      StringUtils.join( Compression.Algorithm.values(), "|").toLowerCase(Locale.ROOT) +
       "\n" +
       "For example:\n" +
       "  hbase " + CompressionTest.class + " file:///tmp/testfile gz\n");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/util/DirectMemoryUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/DirectMemoryUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/DirectMemoryUtils.java
index bc901ab..bd8837d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/DirectMemoryUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/DirectMemoryUtils.java
@@ -25,6 +25,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.List;
+import java.util.Locale;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -87,7 +88,7 @@ public class DirectMemoryUtils {
     long multiplier = 1; //for the byte case.
     for (String s : arguments) {
       if (s.contains("-XX:MaxDirectMemorySize=")) {
-        String memSize = s.toLowerCase()
+        String memSize = s.toLowerCase(Locale.ROOT)
             .replace("-xx:maxdirectmemorysize=", "").trim();
 
         if (memSize.contains("k")) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index d0443b8..74dad43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -35,6 +35,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -1885,7 +1886,7 @@ public abstract class FSUtils {
           return false;
         }
 
-        if (!regionName.toLowerCase().matches("[0-9a-f]+")) {
+        if (!regionName.toLowerCase(Locale.ROOT).matches("[0-9a-f]+")) {
           return false;
         }
         return true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 4311ca5..feb75b7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -3759,7 +3760,7 @@ public class HBaseFsck extends Configured {
           errors.progress();
           String encodedName = regionDir.getPath().getName();
           // ignore directories that aren't hexadecimal
-          if (!encodedName.toLowerCase().matches("[0-9a-f]+")) {
+          if (!encodedName.toLowerCase(Locale.ROOT).matches("[0-9a-f]+")) {
             continue;
           }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
index 983d49c..e6b746c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
@@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Map.Entry;
 import java.util.Set;
 
@@ -103,8 +104,8 @@ public abstract class ServerCommandLine extends Configured implements Tool {
 
       nextEnv:
       for (Entry<String, String> entry : System.getenv().entrySet()) {
-        String key = entry.getKey().toLowerCase();
-        String value = entry.getValue().toLowerCase();
+        String key = entry.getKey().toLowerCase(Locale.ROOT);
+        String value = entry.getValue().toLowerCase(Locale.ROOT);
         // exclude variables which may contain skip words
         for(String skipWord : skipWords) {
           if (key.contains(skipWord) || value.contains(skipWord))

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/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 6a144a0..2b462c9 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
@@ -31,6 +31,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.LinkedList;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Random;
@@ -308,8 +309,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
   static boolean checkTable(HBaseAdmin admin, TestOptions opts) throws IOException {
     TableName tableName = TableName.valueOf(opts.tableName);
     boolean needsDelete = false, exists = admin.tableExists(tableName);
-    boolean isReadCmd = opts.cmdName.toLowerCase().contains("read")
-      || opts.cmdName.toLowerCase().contains("scan");
+    boolean isReadCmd = opts.cmdName.toLowerCase(Locale.ROOT).contains("read")
+      || opts.cmdName.toLowerCase(Locale.ROOT).contains("scan");
     if (!exists && isReadCmd) {
       throw new IllegalStateException(
         "Must specify an existing table for read commands. Run a write command first.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java
index 0baf5de3..09fbe16 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIPv6NIOServerSocketChannel.java
@@ -24,6 +24,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.nio.channels.ServerSocketChannel;
+import java.util.Locale;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -118,7 +119,7 @@ public class TestIPv6NIOServerSocketChannel {
       //java.net.SocketException: Address family not supported by protocol family
       //or java.net.SocketException: Protocol family not supported
       Assert.assertFalse(ex.getClass().isInstance(BindException.class));
-      Assert.assertTrue(ex.getMessage().toLowerCase().contains("protocol family"));
+      Assert.assertTrue(ex.getMessage().toLowerCase(Locale.ROOT).contains("protocol family"));
       LOG.info("Received expected exception:");
       LOG.info(ex);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
index 9a6314a..c96d5fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
@@ -44,6 +44,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.NavigableMap;
 
@@ -226,8 +227,8 @@ public abstract class MultiTableInputFormatTestBase {
   private void testScan(String start, String stop, String last)
       throws IOException, InterruptedException, ClassNotFoundException {
     String jobName =
-        "Scan" + (start != null ? start.toUpperCase() : "Empty") + "To" +
-            (stop != null ? stop.toUpperCase() : "Empty");
+        "Scan" + (start != null ? start.toUpperCase(Locale.ROOT) : "Empty") + "To" +
+            (stop != null ? stop.toUpperCase(Locale.ROOT) : "Empty");
     LOG.info("Before map/reduce startup - job " + jobName);
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
index 8ed7b29..04b8bd9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.Locale;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
@@ -353,7 +354,7 @@ public class TestLoadIncrementalHFiles {
     // set real family name to upper case in purpose to simulate the case that
     // family name in HFiles is invalid
     HColumnDescriptor family =
-        new HColumnDescriptor(Bytes.toBytes(new String(FAMILY).toUpperCase()));
+        new HColumnDescriptor(Bytes.toBytes(new String(FAMILY).toUpperCase(Locale.ROOT)));
     htd.addFamily(family);
 
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
index a29b53c..e167be9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.NavigableMap;
 
@@ -181,8 +182,8 @@ public abstract class TestTableInputFormatScanBase {
    */
   protected void testScanFromConfiguration(String start, String stop, String last)
   throws IOException, InterruptedException, ClassNotFoundException {
-    String jobName = "ScanFromConfig" + (start != null ? start.toUpperCase() : "Empty") +
-      "To" + (stop != null ? stop.toUpperCase() : "Empty");
+    String jobName = "ScanFromConfig" + (start != null ? start.toUpperCase(Locale.ROOT) : "Empty") +
+      "To" + (stop != null ? stop.toUpperCase(Locale.ROOT) : "Empty");
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.set(TableInputFormat.INPUT_TABLE, Bytes.toString(TABLE_NAME));
     c.set(TableInputFormat.SCAN_COLUMN_FAMILY, Bytes.toString(INPUT_FAMILY));
@@ -218,8 +219,8 @@ public abstract class TestTableInputFormatScanBase {
    */
   protected void testScan(String start, String stop, String last)
   throws IOException, InterruptedException, ClassNotFoundException {
-    String jobName = "Scan" + (start != null ? start.toUpperCase() : "Empty") +
-      "To" + (stop != null ? stop.toUpperCase() : "Empty");
+    String jobName = "Scan" + (start != null ? start.toUpperCase(Locale.ROOT) : "Empty") +
+      "To" + (stop != null ? stop.toUpperCase(Locale.ROOT) : "Empty");
     LOG.info("Before map/reduce startup - job " + jobName);
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
index ead913a..826ffbc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
@@ -24,6 +24,7 @@ import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -142,7 +143,7 @@ public class DataBlockEncodingTool {
       String s = super.toString();
       StringBuilder sb = new StringBuilder();
       sb.append(s.charAt(0));
-      sb.append(s.substring(1).toLowerCase());
+      sb.append(s.substring(1).toLowerCase(Locale.ROOT));
       return sb.toString();
     }
   }
@@ -372,7 +373,7 @@ public class DataBlockEncodingTool {
   private void benchmarkDefaultCompression(int totalSize, byte[] rawBuffer)
       throws IOException {
     benchmarkAlgorithm(compressionAlgorithm,
-        compressionAlgorithmName.toUpperCase(), rawBuffer, 0, totalSize);
+        compressionAlgorithmName.toUpperCase(Locale.ROOT), rawBuffer, 0, totalSize);
   }
 
   /**
@@ -526,7 +527,7 @@ public class DataBlockEncodingTool {
    * @throws IOException
    */
   public void displayStatistics() throws IOException {
-    final String comprAlgo = compressionAlgorithmName.toUpperCase();
+    final String comprAlgo = compressionAlgorithmName.toUpperCase(Locale.ROOT);
     long rawBytes = totalKeyLength + totalPrefixLength + totalValueLength;
 
     System.out.println("Raw data size:");
@@ -694,7 +695,7 @@ public class DataBlockEncodingTool {
     String compressionName = DEFAULT_COMPRESSION.getName();
     if (cmd.hasOption(OPT_ENCODING_ALGORITHM)) {
       compressionName =
-          cmd.getOptionValue(OPT_ENCODING_ALGORITHM).toLowerCase();
+          cmd.getOptionValue(OPT_ENCODING_ALGORITHM).toLowerCase(Locale.ROOT);
     }
     boolean doBenchmark = cmd.hasOption(OPT_MEASURE_THROUGHPUT);
     boolean doVerify = !cmd.hasOption(OPT_OMIT_CORRECTNESS_TEST);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
index 1ca75be..afeb0ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/LoadTestTool.java
@@ -24,6 +24,7 @@ import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Locale;
 import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicReference;
@@ -116,7 +117,7 @@ public class LoadTestTool extends AbstractHBaseTool {
   public static final String OPT_DEFERRED_LOG_FLUSH_USAGE = "Enable deferred log flush.";
 
   public static final String OPT_DATA_BLOCK_ENCODING =
-      HColumnDescriptor.DATA_BLOCK_ENCODING.toLowerCase();
+      HColumnDescriptor.DATA_BLOCK_ENCODING.toLowerCase(Locale.ROOT);
 
   public static final String OPT_INMEMORY = "in_memory";
   public static final String OPT_USAGE_IN_MEMORY = "Tries to keep the HFiles of the CF " +

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HThreadedSelectorServerArgs.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HThreadedSelectorServerArgs.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HThreadedSelectorServerArgs.java
index 5fbde7a..da33cc0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HThreadedSelectorServerArgs.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/HThreadedSelectorServerArgs.java
@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hbase.thrift;
 
+import java.util.Locale;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -79,7 +80,7 @@ public class HThreadedSelectorServerArgs extends TThreadedSelectorServer.Args {
     int acceptQueueSizePerThread = conf.getInt(
         ACCEPT_QUEUE_SIZE_PER_THREAD_CONF_KEY, getAcceptQueueSizePerThread());
     AcceptPolicy acceptPolicy = AcceptPolicy.valueOf(conf.get(
-        ACCEPT_POLICY_CONF_KEY, getAcceptPolicy().toString()).toUpperCase());
+        ACCEPT_POLICY_CONF_KEY, getAcceptPolicy().toString()).toUpperCase(Locale.ROOT));
 
     super.selectorThreads(selectorThreads)
          .workerThreads(workerThreads)

http://git-wip-us.apache.org/repos/asf/hbase/blob/e26800c0/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
index 0340ccf..6c2b7b5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.thrift;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -55,7 +56,7 @@ public class ThriftUtilities {
   static public HColumnDescriptor colDescFromThrift(ColumnDescriptor in)
       throws IllegalArgument {
     Compression.Algorithm comp =
-      Compression.getCompressionAlgorithmByName(in.compression.toLowerCase());
+      Compression.getCompressionAlgorithmByName(in.compression.toLowerCase(Locale.ROOT));
     BloomType bt =
       BloomType.valueOf(in.bloomFilterType);
 


[5/8] hbase git commit: HBASE-15931 Add log for long-running tasks in AsyncProcess

Posted by ap...@apache.org.
HBASE-15931 Add log for long-running tasks in AsyncProcess

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: fee078d5ada26c612c87b0d8c437ff9b488bea0f
Parents: 4678c5d
Author: Yu Li <li...@apache.org>
Authored: Thu Jun 2 12:00:42 2016 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 15:04:18 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncProcess.java       | 39 ++++++++++++++++++--
 1 file changed, 35 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fee078d5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index 217eabb..ad6a8d8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
 import org.cloudera.htrace.Trace;
 
 import com.google.common.base.Preconditions;
@@ -105,6 +104,12 @@ class AsyncProcess<CResult> {
   public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
       "hbase.client.start.log.errors.counter";
   public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 9;
+  
+  private final int thresholdToLogUndoneTaskDetails;
+  private static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS =
+      "hbase.client.threshold.log.details";
+  private static final int DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = 10;
+  private final int THRESHOLD_TO_LOG_REGION_DETAILS = 2;
 
   protected static final AtomicLong COUNTER = new AtomicLong();
   protected final long id;
@@ -247,6 +252,9 @@ class AsyncProcess<CResult> {
     this.startLogErrorsCnt =
         conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
 
+    this.thresholdToLogUndoneTaskDetails = conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
+      DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS);
+
     if (this.maxTotalConcurrentTasks <= 0) {
       throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
     }
@@ -321,7 +329,7 @@ class AsyncProcess<CResult> {
       }
 
       // Wait until there is at least one slot for a new task.
-      waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1);
+      waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, tableName.getNameAsString());
 
       // Remember the previous decisions about regions or region servers we put in the
       //  final multi.
@@ -1020,7 +1028,8 @@ class AsyncProcess<CResult> {
   /**
    * Wait until the async does not have more than max tasks in progress.
    */
-  private void waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
+  private void waitForMaximumCurrentTasks(int max, String tableName)
+      throws InterruptedIOException {
     long lastLog = EnvironmentEdgeManager.currentTimeMillis();
     long currentTasksDone = this.tasksDone.get();
 
@@ -1032,6 +1041,9 @@ class AsyncProcess<CResult> {
             + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
             ", currentTasksDone=" + currentTasksDone + ", retries=" + retriesCnt.get() +
             " hasError=" + hasError() + ", tableName=" + tableName);
+        if (getCurrentTasksCount() <= thresholdToLogUndoneTaskDetails) {
+          logDetailsOfUndoneTasks(getCurrentTasksCount());
+        }
       }
       waitForNextTaskDone(currentTasksDone);
       currentTasksDone = this.tasksDone.get();
@@ -1046,7 +1058,7 @@ class AsyncProcess<CResult> {
    * Wait until all tasks are executed, successfully or not.
    */
   public void waitUntilDone() throws InterruptedIOException {
-    waitForMaximumCurrentTasks(0);
+    waitForMaximumCurrentTasks(0, null);
   }
 
 
@@ -1123,4 +1135,23 @@ class AsyncProcess<CResult> {
   protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
     return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout, this.numTries);
   }
+
+  private void logDetailsOfUndoneTasks(long taskInProgress) {
+    ArrayList<ServerName> servers = new ArrayList<ServerName>();
+    for (Map.Entry<ServerName, AtomicInteger> entry : taskCounterPerServer.entrySet()) {
+      if (entry.getValue().get() > 0) {
+        servers.add(entry.getKey());
+      }
+    }
+    LOG.info("Left over " + taskInProgress + " task(s) are processed on server(s): " + servers);
+    if (taskInProgress <= THRESHOLD_TO_LOG_REGION_DETAILS) {
+      ArrayList<String> regions = new ArrayList<String>();
+      for (Map.Entry<byte[], AtomicInteger> entry : taskCounterPerRegion.entrySet()) {
+        if (entry.getValue().get() > 0) {
+          regions.add(Bytes.toString(entry.getKey()));
+        }
+      }
+      LOG.info("Regions against which left over task(s) are processed: " + regions);
+    }
+  }
 }


[8/8] hbase git commit: HBASE-15803 ZooKeeperWatcher's constructor can leak a ZooKeeper instance with throwing ZooKeeperConnectionException when canCreateBaseZNode is true

Posted by ap...@apache.org.
HBASE-15803 ZooKeeperWatcher's constructor can leak a ZooKeeper instance with throwing ZooKeeperConnectionException when canCreateBaseZNode is true


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

Branch: refs/heads/0.98
Commit: 95694adc5f63a5dfe028716f301670991d831a84
Parents: e26800c
Author: tedyu <yu...@gmail.com>
Authored: Tue Jun 7 09:14:01 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 15:37:21 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java | 12 +++++++++++-
 1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/95694adc/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 be79e05..57347e6 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
@@ -171,7 +171,17 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
     this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier);
     pendingWatcher.prepare(this);
     if (canCreateBaseZNode) {
-      createBaseZNodes();
+      try {
+        createBaseZNodes();
+      } catch (ZooKeeperConnectionException zce) {
+        try {
+          this.recoverableZooKeeper.close();
+        } catch (InterruptedException ie) {
+          LOG.debug("Encountered InterruptedException when closing " + this.recoverableZooKeeper);
+          Thread.currentThread().interrupt();
+        }
+        throw zce;
+      }
     }
   }
 


[2/8] hbase git commit: HBASE-15615 Wrong sleep time when RegionServerCallable need retry (Guanghao Zhang)

Posted by ap...@apache.org.
HBASE-15615 Wrong sleep time when RegionServerCallable need retry (Guanghao Zhang)

Conflicts:
	hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionAdminServiceCallable.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java

Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 2d3ef97bcd9837223f6866928d336e54e993726a
Parents: fb1995f
Author: Mikhail Antonov <an...@apache.org>
Authored: Sun May 15 20:49:00 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 14:18:04 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/ConnectionUtils.java    |  3 ++
 .../hbase/client/RegionServerCallable.java      |  3 +-
 .../hadoop/hbase/client/RpcRetryingCaller.java  |  4 +-
 .../hbase/client/TestConnectionUtils.java       | 20 ++++++++++
 .../org/apache/hadoop/hbase/client/TestHCM.java | 40 +++++---------------
 5 files changed, 35 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d3ef97b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 0823bbb..5eb1047 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -41,6 +41,9 @@ public class ConnectionUtils {
     if (ntries >= HConstants.RETRY_BACKOFF.length) {
       ntries = HConstants.RETRY_BACKOFF.length - 1;
     }
+    if (ntries < 0) {
+      ntries = 0;
+    }
 
     long normalPause = pause * HConstants.RETRY_BACKOFF[ntries];
     long jitter =  (long)(normalPause * RANDOM.nextFloat() * 0.01f); // 1% possible jitter

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d3ef97b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
index 65cd2f3..d0a399a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java
@@ -136,8 +136,7 @@ public abstract class RegionServerCallable<T> implements RetryingCallable<T> {
 
   @Override
   public long sleep(long pause, int tries) {
-    // Tries hasn't been bumped up yet so we use "tries + 1" to get right pause time
-    long sleep = ConnectionUtils.getPauseTime(pause, tries + 1);
+    long sleep = ConnectionUtils.getPauseTime(pause, tries);
     if (sleep < MIN_WAIT_DEAD_SERVER
         && (location == null || getConnection().isDeadServer(location.getServerName()))) {
       sleep = ConnectionUtils.addJitter(MIN_WAIT_DEAD_SERVER, 0.10f);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d3ef97b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
index 95fbb92..8b1713f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCaller.java
@@ -132,8 +132,8 @@ public class RpcRetryingCaller<T> {
         }
         // If the server is dead, we need to wait a little before retrying, to give
         //  a chance to the regions to be
-        // tries hasn't been bumped up yet so we use "tries + 1" to get right pause time
-        expectedSleep = callable.sleep(pause, tries + 1);
+        // get right pause time, start by RETRY_BACKOFF[0] * pause
+        expectedSleep = callable.sleep(pause, tries);
 
         // If, after the planned sleep, there won't be enough time left, we stop now.
         long duration = singleCallDuration(expectedSleep);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d3ef97b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java
index 649d674..3d449ae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestConnectionUtils.java
@@ -19,6 +19,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -53,4 +54,23 @@ public class TestConnectionUtils {
     assertTrue(retyTimeSet.size() > (retries.length * 0.80));
   }
 
+  @Test
+  public void testGetPauseTime() {
+    long pauseTime;
+    long baseTime = 100;
+    pauseTime = ConnectionUtils.getPauseTime(baseTime, -1);
+    assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[0]));
+    assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[0] * 1.01f));
+
+    for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
+      pauseTime = ConnectionUtils.getPauseTime(baseTime, i);
+      assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[i]));
+      assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[i] * 1.01f));
+    }
+
+    int length = HConstants.RETRY_BACKOFF.length;
+    pauseTime = ConnectionUtils.getPauseTime(baseTime, length);
+    assertTrue(pauseTime >= (baseTime * HConstants.RETRY_BACKOFF[length - 1]));
+    assertTrue(pauseTime <= (baseTime * HConstants.RETRY_BACKOFF[length - 1] * 1.01f));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d3ef97b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 0ab204d..1f1c9e4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -866,7 +866,7 @@ public class TestHCM {
     conn.close();
   }
 
-  @Ignore ("Test presumes RETRY_BACKOFF will never change; it has") @Test
+  @Test
   public void testErrorBackoffTimeCalculation() throws Exception {
     // TODO: This test would seem to presume hardcoded RETRY_BACKOFF which it should not.
     final long ANY_PAUSE = 100;
@@ -887,46 +887,24 @@ public class TestHCM {
 
       // Check some backoff values from HConstants sequence.
       tracker.reportServerError(location);
-      assertEqualsWithJitter(ANY_PAUSE, tracker.calculateBackoffTime(location, ANY_PAUSE));
+      assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[0],
+        tracker.calculateBackoffTime(location, ANY_PAUSE));
       tracker.reportServerError(location);
       tracker.reportServerError(location);
       tracker.reportServerError(location);
-      assertEqualsWithJitter(ANY_PAUSE * 5, tracker.calculateBackoffTime(location, ANY_PAUSE));
+      assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[3],
+        tracker.calculateBackoffTime(location, ANY_PAUSE));
 
       // All of this shouldn't affect backoff for different location.
 
       assertEquals(0, tracker.calculateBackoffTime(diffLocation, ANY_PAUSE));
       tracker.reportServerError(diffLocation);
-      assertEqualsWithJitter(ANY_PAUSE, tracker.calculateBackoffTime(diffLocation, ANY_PAUSE));
-
-      // But should still work for a different region in the same location.
-      HRegionInfo ri2 = new HRegionInfo(TABLE_NAME2);
-      HRegionLocation diffRegion = new HRegionLocation(ri2, location.getServerName());
-      assertEqualsWithJitter(ANY_PAUSE * 5, tracker.calculateBackoffTime(diffRegion, ANY_PAUSE));
+      assertEqualsWithJitter(ANY_PAUSE * HConstants.RETRY_BACKOFF[0],
+        tracker.calculateBackoffTime(diffLocation, ANY_PAUSE));
 
       // Check with different base.
-      assertEqualsWithJitter(ANY_PAUSE * 10,
-          tracker.calculateBackoffTime(location, ANY_PAUSE * 2));
-
-      // See that time from last error is taken into account. Time shift is applied after jitter,
-      // so pass the original expected backoff as the base for jitter.
-      long timeShift = (long)(ANY_PAUSE * 0.5);
-      timeMachine.setValue(timeBase + timeShift);
-      assertEqualsWithJitter((ANY_PAUSE * 5) - timeShift,
-        tracker.calculateBackoffTime(location, ANY_PAUSE), ANY_PAUSE * 2);
-
-      // However we should not go into negative.
-      timeMachine.setValue(timeBase + ANY_PAUSE * 100);
-      assertEquals(0, tracker.calculateBackoffTime(location, ANY_PAUSE));
-
-      // We also should not go over the boundary; last retry would be on it.
-      long timeLeft = (long)(ANY_PAUSE * 0.5);
-      timeMachine.setValue(timeBase + largeAmountOfTime - timeLeft);
-      assertTrue(tracker.canRetryMore(1));
-      tracker.reportServerError(location);
-      assertEquals(timeLeft, tracker.calculateBackoffTime(location, ANY_PAUSE));
-      timeMachine.setValue(timeBase + largeAmountOfTime);
-      assertFalse(tracker.canRetryMore(1));
+      assertEqualsWithJitter(ANY_PAUSE * 2 * HConstants.RETRY_BACKOFF[3],
+        tracker.calculateBackoffTime(location, ANY_PAUSE * 2));
     } finally {
       EnvironmentEdgeManager.reset();
     }


[4/8] hbase git commit: HBASE-15890 Allow setting cacheBlocks for TScan

Posted by ap...@apache.org.
HBASE-15890 Allow setting cacheBlocks for TScan

Signed-off-by: stack <st...@apache.org>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/0.98
Commit: 4678c5d931a47fc0a04b560adf7da979687b8ad8
Parents: 458378d
Author: Ashu Pachauri <as...@gmail.com>
Authored: Fri May 27 17:54:20 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jun 7 14:50:46 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/thrift/ThriftServerRunner.java |   3 +
 .../hadoop/hbase/thrift/generated/Hbase.java    | 104 ++++++++---------
 .../hbase/thrift/generated/TRowResult.java      |   4 +-
 .../hadoop/hbase/thrift/generated/TScan.java    | 108 +++++++++++++++++-
 .../hadoop/hbase/thrift2/ThriftUtilities.java   |   4 +
 .../hadoop/hbase/thrift2/generated/TAppend.java |   4 +-
 .../hadoop/hbase/thrift2/generated/TDelete.java |   4 +-
 .../hadoop/hbase/thrift2/generated/TGet.java    |   4 +-
 .../hbase/thrift2/generated/TIncrement.java     |   4 +-
 .../hadoop/hbase/thrift2/generated/TPut.java    |   4 +-
 .../hadoop/hbase/thrift2/generated/TScan.java   | 112 ++++++++++++++++++-
 .../org/apache/hadoop/hbase/thrift/Hbase.thrift |   3 +-
 .../apache/hadoop/hbase/thrift2/hbase.thrift    |   1 +
 13 files changed, 284 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index 804ad31..496b51c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -1382,6 +1382,9 @@ public class ThriftServerRunner implements Runnable {
         if (tScan.isSetReversed()) {
           scan.setReversed(tScan.isReversed());
         }
+        if (tScan.isSetCacheBlocks()) {
+          scan.setCacheBlocks(tScan.isCacheBlocks());
+        }
         return addScanner(table.getScanner(scan), tScan.sortColumns);
       } catch (IOException e) {
         LOG.warn(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
index bd5bd52..2eefad4 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
@@ -10061,7 +10061,7 @@ public class Hbase {
                   struct.success = new HashMap<ByteBuffer,ColumnDescriptor>(2*_map42.size);
                   for (int _i43 = 0; _i43 < _map42.size; ++_i43)
                   {
-                    ByteBuffer _key44; // optional
+                    ByteBuffer _key44; // required
                     ColumnDescriptor _val45; // required
                     _key44 = iprot.readBinary();
                     _val45 = new ColumnDescriptor();
@@ -10167,7 +10167,7 @@ public class Hbase {
             struct.success = new HashMap<ByteBuffer,ColumnDescriptor>(2*_map48.size);
             for (int _i49 = 0; _i49 < _map48.size; ++_i49)
             {
-              ByteBuffer _key50; // optional
+              ByteBuffer _key50; // required
               ColumnDescriptor _val51; // required
               _key50 = iprot.readBinary();
               _val51 = new ColumnDescriptor();
@@ -13642,7 +13642,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map68.size);
                   for (int _i69 = 0; _i69 < _map68.size; ++_i69)
                   {
-                    ByteBuffer _key70; // optional
+                    ByteBuffer _key70; // required
                     ByteBuffer _val71; // required
                     _key70 = iprot.readBinary();
                     _val71 = iprot.readBinary();
@@ -13772,7 +13772,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map74.size);
             for (int _i75 = 0; _i75 < _map74.size; ++_i75)
             {
-              ByteBuffer _key76; // optional
+              ByteBuffer _key76; // required
               ByteBuffer _val77; // required
               _key76 = iprot.readBinary();
               _val77 = iprot.readBinary();
@@ -15083,7 +15083,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map86.size);
                   for (int _i87 = 0; _i87 < _map86.size; ++_i87)
                   {
-                    ByteBuffer _key88; // optional
+                    ByteBuffer _key88; // required
                     ByteBuffer _val89; // required
                     _key88 = iprot.readBinary();
                     _val89 = iprot.readBinary();
@@ -15226,7 +15226,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map92.size);
             for (int _i93 = 0; _i93 < _map92.size; ++_i93)
             {
-              ByteBuffer _key94; // optional
+              ByteBuffer _key94; // required
               ByteBuffer _val95; // required
               _key94 = iprot.readBinary();
               _val95 = iprot.readBinary();
@@ -16635,7 +16635,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map104.size);
                   for (int _i105 = 0; _i105 < _map104.size; ++_i105)
                   {
-                    ByteBuffer _key106; // optional
+                    ByteBuffer _key106; // required
                     ByteBuffer _val107; // required
                     _key106 = iprot.readBinary();
                     _val107 = iprot.readBinary();
@@ -16791,7 +16791,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map110.size);
             for (int _i111 = 0; _i111 < _map110.size; ++_i111)
             {
-              ByteBuffer _key112; // optional
+              ByteBuffer _key112; // required
               ByteBuffer _val113; // required
               _key112 = iprot.readBinary();
               _val113 = iprot.readBinary();
@@ -17888,7 +17888,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map122.size);
                   for (int _i123 = 0; _i123 < _map122.size; ++_i123)
                   {
-                    ByteBuffer _key124; // optional
+                    ByteBuffer _key124; // required
                     ByteBuffer _val125; // required
                     _key124 = iprot.readBinary();
                     _val125 = iprot.readBinary();
@@ -18003,7 +18003,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map128.size);
             for (int _i129 = 0; _i129 < _map128.size; ++_i129)
             {
-              ByteBuffer _key130; // optional
+              ByteBuffer _key130; // required
               ByteBuffer _val131; // required
               _key130 = iprot.readBinary();
               _val131 = iprot.readBinary();
@@ -19232,7 +19232,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map143.size);
                   for (int _i144 = 0; _i144 < _map143.size; ++_i144)
                   {
-                    ByteBuffer _key145; // optional
+                    ByteBuffer _key145; // required
                     ByteBuffer _val146; // required
                     _key145 = iprot.readBinary();
                     _val146 = iprot.readBinary();
@@ -19384,7 +19384,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map154.size);
             for (int _i155 = 0; _i155 < _map154.size; ++_i155)
             {
-              ByteBuffer _key156; // optional
+              ByteBuffer _key156; // required
               ByteBuffer _val157; // required
               _key156 = iprot.readBinary();
               _val157 = iprot.readBinary();
@@ -20583,7 +20583,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map166.size);
                   for (int _i167 = 0; _i167 < _map166.size; ++_i167)
                   {
-                    ByteBuffer _key168; // optional
+                    ByteBuffer _key168; // required
                     ByteBuffer _val169; // required
                     _key168 = iprot.readBinary();
                     _val169 = iprot.readBinary();
@@ -20711,7 +20711,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map172.size);
             for (int _i173 = 0; _i173 < _map172.size; ++_i173)
             {
-              ByteBuffer _key174; // optional
+              ByteBuffer _key174; // required
               ByteBuffer _val175; // required
               _key174 = iprot.readBinary();
               _val175 = iprot.readBinary();
@@ -22030,7 +22030,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map187.size);
                   for (int _i188 = 0; _i188 < _map187.size; ++_i188)
                   {
-                    ByteBuffer _key189; // optional
+                    ByteBuffer _key189; // required
                     ByteBuffer _val190; // required
                     _key189 = iprot.readBinary();
                     _val190 = iprot.readBinary();
@@ -22195,7 +22195,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map198.size);
             for (int _i199 = 0; _i199 < _map198.size; ++_i199)
             {
-              ByteBuffer _key200; // optional
+              ByteBuffer _key200; // required
               ByteBuffer _val201; // required
               _key200 = iprot.readBinary();
               _val201 = iprot.readBinary();
@@ -23312,7 +23312,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map213.size);
                   for (int _i214 = 0; _i214 < _map213.size; ++_i214)
                   {
-                    ByteBuffer _key215; // optional
+                    ByteBuffer _key215; // required
                     ByteBuffer _val216; // required
                     _key215 = iprot.readBinary();
                     _val216 = iprot.readBinary();
@@ -23449,7 +23449,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map224.size);
             for (int _i225 = 0; _i225 < _map224.size; ++_i225)
             {
-              ByteBuffer _key226; // optional
+              ByteBuffer _key226; // required
               ByteBuffer _val227; // required
               _key226 = iprot.readBinary();
               _val227 = iprot.readBinary();
@@ -24698,7 +24698,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map242.size);
                   for (int _i243 = 0; _i243 < _map242.size; ++_i243)
                   {
-                    ByteBuffer _key244; // optional
+                    ByteBuffer _key244; // required
                     ByteBuffer _val245; // required
                     _key244 = iprot.readBinary();
                     _val245 = iprot.readBinary();
@@ -24872,7 +24872,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map258.size);
             for (int _i259 = 0; _i259 < _map258.size; ++_i259)
             {
-              ByteBuffer _key260; // optional
+              ByteBuffer _key260; // required
               ByteBuffer _val261; // required
               _key260 = iprot.readBinary();
               _val261 = iprot.readBinary();
@@ -26091,7 +26091,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map273.size);
                   for (int _i274 = 0; _i274 < _map273.size; ++_i274)
                   {
-                    ByteBuffer _key275; // optional
+                    ByteBuffer _key275; // required
                     ByteBuffer _val276; // required
                     _key275 = iprot.readBinary();
                     _val276 = iprot.readBinary();
@@ -26241,7 +26241,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map284.size);
             for (int _i285 = 0; _i285 < _map284.size; ++_i285)
             {
-              ByteBuffer _key286; // optional
+              ByteBuffer _key286; // required
               ByteBuffer _val287; // required
               _key286 = iprot.readBinary();
               _val287 = iprot.readBinary();
@@ -27580,7 +27580,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map302.size);
                   for (int _i303 = 0; _i303 < _map302.size; ++_i303)
                   {
-                    ByteBuffer _key304; // optional
+                    ByteBuffer _key304; // required
                     ByteBuffer _val305; // required
                     _key304 = iprot.readBinary();
                     _val305 = iprot.readBinary();
@@ -27767,7 +27767,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map318.size);
             for (int _i319 = 0; _i319 < _map318.size; ++_i319)
             {
-              ByteBuffer _key320; // optional
+              ByteBuffer _key320; // required
               ByteBuffer _val321; // required
               _key320 = iprot.readBinary();
               _val321 = iprot.readBinary();
@@ -28997,7 +28997,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map333.size);
                   for (int _i334 = 0; _i334 < _map333.size; ++_i334)
                   {
-                    ByteBuffer _key335; // optional
+                    ByteBuffer _key335; // required
                     ByteBuffer _val336; // required
                     _key335 = iprot.readBinary();
                     _val336 = iprot.readBinary();
@@ -29150,7 +29150,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map344.size);
             for (int _i345 = 0; _i345 < _map344.size; ++_i345)
             {
-              ByteBuffer _key346; // optional
+              ByteBuffer _key346; // required
               ByteBuffer _val347; // required
               _key346 = iprot.readBinary();
               _val347 = iprot.readBinary();
@@ -30430,7 +30430,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map351.size);
                   for (int _i352 = 0; _i352 < _map351.size; ++_i352)
                   {
-                    ByteBuffer _key353; // optional
+                    ByteBuffer _key353; // required
                     ByteBuffer _val354; // required
                     _key353 = iprot.readBinary();
                     _val354 = iprot.readBinary();
@@ -30596,7 +30596,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map362.size);
             for (int _i363 = 0; _i363 < _map362.size; ++_i363)
             {
-              ByteBuffer _key364; // optional
+              ByteBuffer _key364; // required
               ByteBuffer _val365; // required
               _key364 = iprot.readBinary();
               _val365 = iprot.readBinary();
@@ -31662,7 +31662,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map369.size);
                   for (int _i370 = 0; _i370 < _map369.size; ++_i370)
                   {
-                    ByteBuffer _key371; // optional
+                    ByteBuffer _key371; // required
                     ByteBuffer _val372; // required
                     _key371 = iprot.readBinary();
                     _val372 = iprot.readBinary();
@@ -31800,7 +31800,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map380.size);
             for (int _i381 = 0; _i381 < _map380.size; ++_i381)
             {
-              ByteBuffer _key382; // optional
+              ByteBuffer _key382; // required
               ByteBuffer _val383; // required
               _key382 = iprot.readBinary();
               _val383 = iprot.readBinary();
@@ -32968,7 +32968,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map387.size);
                   for (int _i388 = 0; _i388 < _map387.size; ++_i388)
                   {
-                    ByteBuffer _key389; // optional
+                    ByteBuffer _key389; // required
                     ByteBuffer _val390; // required
                     _key389 = iprot.readBinary();
                     _val390 = iprot.readBinary();
@@ -33119,7 +33119,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map398.size);
             for (int _i399 = 0; _i399 < _map398.size; ++_i399)
             {
-              ByteBuffer _key400; // optional
+              ByteBuffer _key400; // required
               ByteBuffer _val401; // required
               _key400 = iprot.readBinary();
               _val401 = iprot.readBinary();
@@ -35603,7 +35603,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map402.size);
                   for (int _i403 = 0; _i403 < _map402.size; ++_i403)
                   {
-                    ByteBuffer _key404; // optional
+                    ByteBuffer _key404; // required
                     ByteBuffer _val405; // required
                     _key404 = iprot.readBinary();
                     _val405 = iprot.readBinary();
@@ -35733,7 +35733,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map408.size);
             for (int _i409 = 0; _i409 < _map408.size; ++_i409)
             {
-              ByteBuffer _key410; // optional
+              ByteBuffer _key410; // required
               ByteBuffer _val411; // required
               _key410 = iprot.readBinary();
               _val411 = iprot.readBinary();
@@ -36885,7 +36885,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map412.size);
                   for (int _i413 = 0; _i413 < _map412.size; ++_i413)
                   {
-                    ByteBuffer _key414; // optional
+                    ByteBuffer _key414; // required
                     ByteBuffer _val415; // required
                     _key414 = iprot.readBinary();
                     _val415 = iprot.readBinary();
@@ -37028,7 +37028,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map418.size);
             for (int _i419 = 0; _i419 < _map418.size; ++_i419)
             {
-              ByteBuffer _key420; // optional
+              ByteBuffer _key420; // required
               ByteBuffer _val421; // required
               _key420 = iprot.readBinary();
               _val421 = iprot.readBinary();
@@ -37966,7 +37966,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map422.size);
                   for (int _i423 = 0; _i423 < _map422.size; ++_i423)
                   {
-                    ByteBuffer _key424; // optional
+                    ByteBuffer _key424; // required
                     ByteBuffer _val425; // required
                     _key424 = iprot.readBinary();
                     _val425 = iprot.readBinary();
@@ -38081,7 +38081,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map428.size);
             for (int _i429 = 0; _i429 < _map428.size; ++_i429)
             {
-              ByteBuffer _key430; // optional
+              ByteBuffer _key430; // required
               ByteBuffer _val431; // required
               _key430 = iprot.readBinary();
               _val431 = iprot.readBinary();
@@ -40652,7 +40652,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map440.size);
                   for (int _i441 = 0; _i441 < _map440.size; ++_i441)
                   {
-                    ByteBuffer _key442; // optional
+                    ByteBuffer _key442; // required
                     ByteBuffer _val443; // required
                     _key442 = iprot.readBinary();
                     _val443 = iprot.readBinary();
@@ -40780,7 +40780,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map446.size);
             for (int _i447 = 0; _i447 < _map446.size; ++_i447)
             {
-              ByteBuffer _key448; // optional
+              ByteBuffer _key448; // required
               ByteBuffer _val449; // required
               _key448 = iprot.readBinary();
               _val449 = iprot.readBinary();
@@ -41712,7 +41712,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map450.size);
                   for (int _i451 = 0; _i451 < _map450.size; ++_i451)
                   {
-                    ByteBuffer _key452; // optional
+                    ByteBuffer _key452; // required
                     ByteBuffer _val453; // required
                     _key452 = iprot.readBinary();
                     _val453 = iprot.readBinary();
@@ -41828,7 +41828,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map456.size);
             for (int _i457 = 0; _i457 < _map456.size; ++_i457)
             {
-              ByteBuffer _key458; // optional
+              ByteBuffer _key458; // required
               ByteBuffer _val459; // required
               _key458 = iprot.readBinary();
               _val459 = iprot.readBinary();
@@ -43015,7 +43015,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map463.size);
                   for (int _i464 = 0; _i464 < _map463.size; ++_i464)
                   {
-                    ByteBuffer _key465; // optional
+                    ByteBuffer _key465; // required
                     ByteBuffer _val466; // required
                     _key465 = iprot.readBinary();
                     _val466 = iprot.readBinary();
@@ -43167,7 +43167,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map474.size);
             for (int _i475 = 0; _i475 < _map474.size; ++_i475)
             {
-              ByteBuffer _key476; // optional
+              ByteBuffer _key476; // required
               ByteBuffer _val477; // required
               _key476 = iprot.readBinary();
               _val477 = iprot.readBinary();
@@ -44470,7 +44470,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map481.size);
                   for (int _i482 = 0; _i482 < _map481.size; ++_i482)
                   {
-                    ByteBuffer _key483; // optional
+                    ByteBuffer _key483; // required
                     ByteBuffer _val484; // required
                     _key483 = iprot.readBinary();
                     _val484 = iprot.readBinary();
@@ -44637,7 +44637,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map492.size);
             for (int _i493 = 0; _i493 < _map492.size; ++_i493)
             {
-              ByteBuffer _key494; // optional
+              ByteBuffer _key494; // required
               ByteBuffer _val495; // required
               _key494 = iprot.readBinary();
               _val495 = iprot.readBinary();
@@ -45812,7 +45812,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map499.size);
                   for (int _i500 = 0; _i500 < _map499.size; ++_i500)
                   {
-                    ByteBuffer _key501; // optional
+                    ByteBuffer _key501; // required
                     ByteBuffer _val502; // required
                     _key501 = iprot.readBinary();
                     _val502 = iprot.readBinary();
@@ -45964,7 +45964,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map510.size);
             for (int _i511 = 0; _i511 < _map510.size; ++_i511)
             {
-              ByteBuffer _key512; // optional
+              ByteBuffer _key512; // required
               ByteBuffer _val513; // required
               _key512 = iprot.readBinary();
               _val513 = iprot.readBinary();
@@ -47253,7 +47253,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map517.size);
                   for (int _i518 = 0; _i518 < _map517.size; ++_i518)
                   {
-                    ByteBuffer _key519; // optional
+                    ByteBuffer _key519; // required
                     ByteBuffer _val520; // required
                     _key519 = iprot.readBinary();
                     _val520 = iprot.readBinary();
@@ -47418,7 +47418,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map528.size);
             for (int _i529 = 0; _i529 < _map528.size; ++_i529)
             {
-              ByteBuffer _key530; // optional
+              ByteBuffer _key530; // required
               ByteBuffer _val531; // required
               _key530 = iprot.readBinary();
               _val531 = iprot.readBinary();
@@ -48823,7 +48823,7 @@ public class Hbase {
                   struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map535.size);
                   for (int _i536 = 0; _i536 < _map535.size; ++_i536)
                   {
-                    ByteBuffer _key537; // optional
+                    ByteBuffer _key537; // required
                     ByteBuffer _val538; // required
                     _key537 = iprot.readBinary();
                     _val538 = iprot.readBinary();
@@ -49003,7 +49003,7 @@ public class Hbase {
             struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map546.size);
             for (int _i547 = 0; _i547 < _map546.size; ++_i547)
             {
-              ByteBuffer _key548; // optional
+              ByteBuffer _key548; // required
               ByteBuffer _val549; // required
               _key548 = iprot.readBinary();
               _val549 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
index 66757d6..3bb244d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
@@ -562,7 +562,7 @@ public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResul
                 struct.columns = new HashMap<ByteBuffer,TCell>(2*_map8.size);
                 for (int _i9 = 0; _i9 < _map8.size; ++_i9)
                 {
-                  ByteBuffer _key10; // optional
+                  ByteBuffer _key10; // required
                   TCell _val11; // required
                   _key10 = iprot.readBinary();
                   _val11 = new TCell();
@@ -710,7 +710,7 @@ public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResul
           struct.columns = new HashMap<ByteBuffer,TCell>(2*_map19.size);
           for (int _i20 = 0; _i20 < _map19.size; ++_i20)
           {
-            ByteBuffer _key21; // optional
+            ByteBuffer _key21; // required
             TCell _val22; // required
             _key21 = iprot.readBinary();
             _val22 = new TCell();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
index 5f5cdc7..197eb32 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
@@ -46,6 +46,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   private static final org.apache.thrift.protocol.TField BATCH_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("batchSize", org.apache.thrift.protocol.TType.I32, (short)7);
   private static final org.apache.thrift.protocol.TField SORT_COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("sortColumns", org.apache.thrift.protocol.TType.BOOL, (short)8);
   private static final org.apache.thrift.protocol.TField REVERSED_FIELD_DESC = new org.apache.thrift.protocol.TField("reversed", org.apache.thrift.protocol.TType.BOOL, (short)9);
+  private static final org.apache.thrift.protocol.TField CACHE_BLOCKS_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheBlocks", org.apache.thrift.protocol.TType.BOOL, (short)10);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -62,6 +63,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   public int batchSize; // optional
   public boolean sortColumns; // optional
   public boolean reversed; // optional
+  public boolean cacheBlocks; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -73,7 +75,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     FILTER_STRING((short)6, "filterString"),
     BATCH_SIZE((short)7, "batchSize"),
     SORT_COLUMNS((short)8, "sortColumns"),
-    REVERSED((short)9, "reversed");
+    REVERSED((short)9, "reversed"),
+    CACHE_BLOCKS((short)10, "cacheBlocks");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -106,6 +109,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
           return SORT_COLUMNS;
         case 9: // REVERSED
           return REVERSED;
+        case 10: // CACHE_BLOCKS
+          return CACHE_BLOCKS;
         default:
           return null;
       }
@@ -151,8 +156,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   private static final int __BATCHSIZE_ISSET_ID = 2;
   private static final int __SORTCOLUMNS_ISSET_ID = 3;
   private static final int __REVERSED_ISSET_ID = 4;
+  private static final int __CACHEBLOCKS_ISSET_ID = 5;
   private byte __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.TIMESTAMP,_Fields.COLUMNS,_Fields.CACHING,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.SORT_COLUMNS,_Fields.REVERSED};
+  private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.TIMESTAMP,_Fields.COLUMNS,_Fields.CACHING,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.SORT_COLUMNS,_Fields.REVERSED,_Fields.CACHE_BLOCKS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -175,6 +181,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.REVERSED, new org.apache.thrift.meta_data.FieldMetaData("reversed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CACHE_BLOCKS, new org.apache.thrift.meta_data.FieldMetaData("cacheBlocks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TScan.class, metaDataMap);
   }
@@ -208,6 +216,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     this.batchSize = other.batchSize;
     this.sortColumns = other.sortColumns;
     this.reversed = other.reversed;
+    this.cacheBlocks = other.cacheBlocks;
   }
 
   public TScan deepCopy() {
@@ -230,6 +239,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     this.sortColumns = false;
     setReversedIsSet(false);
     this.reversed = false;
+    setCacheBlocksIsSet(false);
+    this.cacheBlocks = false;
   }
 
   public byte[] getStartRow() {
@@ -488,6 +499,29 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REVERSED_ISSET_ID, value);
   }
 
+  public boolean isCacheBlocks() {
+    return this.cacheBlocks;
+  }
+
+  public TScan setCacheBlocks(boolean cacheBlocks) {
+    this.cacheBlocks = cacheBlocks;
+    setCacheBlocksIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheBlocks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  /** Returns true if field cacheBlocks is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheBlocks() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  public void setCacheBlocksIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case START_ROW:
@@ -562,6 +596,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       }
       break;
 
+    case CACHE_BLOCKS:
+      if (value == null) {
+        unsetCacheBlocks();
+      } else {
+        setCacheBlocks((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -594,6 +636,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     case REVERSED:
       return Boolean.valueOf(isReversed());
 
+    case CACHE_BLOCKS:
+      return Boolean.valueOf(isCacheBlocks());
+
     }
     throw new IllegalStateException();
   }
@@ -623,6 +668,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       return isSetSortColumns();
     case REVERSED:
       return isSetReversed();
+    case CACHE_BLOCKS:
+      return isSetCacheBlocks();
     }
     throw new IllegalStateException();
   }
@@ -721,6 +768,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         return false;
     }
 
+    boolean this_present_cacheBlocks = true && this.isSetCacheBlocks();
+    boolean that_present_cacheBlocks = true && that.isSetCacheBlocks();
+    if (this_present_cacheBlocks || that_present_cacheBlocks) {
+      if (!(this_present_cacheBlocks && that_present_cacheBlocks))
+        return false;
+      if (this.cacheBlocks != that.cacheBlocks)
+        return false;
+    }
+
     return true;
   }
 
@@ -773,6 +829,11 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     if (present_reversed)
       builder.append(reversed);
 
+    boolean present_cacheBlocks = true && (isSetCacheBlocks());
+    builder.append(present_cacheBlocks);
+    if (present_cacheBlocks)
+      builder.append(cacheBlocks);
+
     return builder.toHashCode();
   }
 
@@ -874,6 +935,16 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCacheBlocks()).compareTo(typedOther.isSetCacheBlocks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheBlocks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheBlocks, typedOther.cacheBlocks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -963,6 +1034,12 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       sb.append(this.reversed);
       first = false;
     }
+    if (isSetCacheBlocks()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheBlocks:");
+      sb.append(this.cacheBlocks);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1090,6 +1167,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 10: // CACHE_BLOCKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheBlocks = iprot.readBool();
+              struct.setCacheBlocksIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1165,6 +1250,11 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         oprot.writeBool(struct.reversed);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeFieldBegin(CACHE_BLOCKS_FIELD_DESC);
+        oprot.writeBool(struct.cacheBlocks);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1210,7 +1300,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       if (struct.isSetReversed()) {
         optionals.set(8);
       }
-      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetCacheBlocks()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
       if (struct.isSetStartRow()) {
         oprot.writeBinary(struct.startRow);
       }
@@ -1244,12 +1337,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       if (struct.isSetReversed()) {
         oprot.writeBool(struct.reversed);
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeBool(struct.cacheBlocks);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, TScan struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(9);
+      BitSet incoming = iprot.readBitSet(10);
       if (incoming.get(0)) {
         struct.startRow = iprot.readBinary();
         struct.setStartRowIsSet(true);
@@ -1295,6 +1391,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         struct.reversed = iprot.readBool();
         struct.setReversedIsSet(true);
       }
+      if (incoming.get(9)) {
+        struct.cacheBlocks = iprot.readBool();
+        struct.setCacheBlocksIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
index 2fb17d5..6363db4 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
@@ -425,6 +425,10 @@ public class ThriftUtilities {
       out.setReversed(in.isReversed());
     }
 
+    if (in.isSetCacheBlocks()) {
+      out.setCacheBlocks(in.isCacheBlocks());
+    }
+
     return out;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
index 262cb82..49d0e75 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAppend.java
@@ -770,7 +770,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map91.size);
                 for (int _i92 = 0; _i92 < _map91.size; ++_i92)
                 {
-                  ByteBuffer _key93; // optional
+                  ByteBuffer _key93; // required
                   ByteBuffer _val94; // required
                   _key93 = iprot.readBinary();
                   _val94 = iprot.readBinary();
@@ -939,7 +939,7 @@ public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map102.size);
           for (int _i103 = 0; _i103 < _map102.size; ++_i103)
           {
-            ByteBuffer _key104; // optional
+            ByteBuffer _key104; // required
             ByteBuffer _val105; // required
             _key104 = iprot.readBinary();
             _val105 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
index abbad1c..a4d7777 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
@@ -906,7 +906,7 @@ public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map55.size);
                 for (int _i56 = 0; _i56 < _map55.size; ++_i56)
                 {
-                  ByteBuffer _key57; // optional
+                  ByteBuffer _key57; // required
                   ByteBuffer _val58; // required
                   _key57 = iprot.readBinary();
                   _val58 = iprot.readBinary();
@@ -1094,7 +1094,7 @@ public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map66.size);
           for (int _i67 = 0; _i67 < _map66.size; ++_i67)
           {
-            ByteBuffer _key68; // optional
+            ByteBuffer _key68; // required
             ByteBuffer _val69; // required
             _key68 = iprot.readBinary();
             _val69 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
index 8e242b2..7e93341 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
@@ -1051,7 +1051,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map19.size);
                 for (int _i20 = 0; _i20 < _map19.size; ++_i20)
                 {
-                  ByteBuffer _key21; // optional
+                  ByteBuffer _key21; // required
                   ByteBuffer _val22; // required
                   _key21 = iprot.readBinary();
                   _val22 = iprot.readBinary();
@@ -1273,7 +1273,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map30.size);
           for (int _i31 = 0; _i31 < _map30.size; ++_i31)
           {
-            ByteBuffer _key32; // optional
+            ByteBuffer _key32; // required
             ByteBuffer _val33; // required
             _key32 = iprot.readBinary();
             _val33 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
index 90345aa..358a9fc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
@@ -777,7 +777,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map73.size);
                 for (int _i74 = 0; _i74 < _map73.size; ++_i74)
                 {
-                  ByteBuffer _key75; // optional
+                  ByteBuffer _key75; // required
                   ByteBuffer _val76; // required
                   _key75 = iprot.readBinary();
                   _val76 = iprot.readBinary();
@@ -946,7 +946,7 @@ public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncremen
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map84.size);
           for (int _i85 = 0; _i85 < _map84.size; ++_i85)
           {
-            ByteBuffer _key86; // optional
+            ByteBuffer _key86; // required
             ByteBuffer _val87; // required
             _key86 = iprot.readBinary();
             _val87 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
index 62ddffb..c9a122c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
@@ -871,7 +871,7 @@ public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.i
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map37.size);
                 for (int _i38 = 0; _i38 < _map37.size; ++_i38)
                 {
-                  ByteBuffer _key39; // optional
+                  ByteBuffer _key39; // required
                   ByteBuffer _val40; // required
                   _key39 = iprot.readBinary();
                   _val40 = iprot.readBinary();
@@ -1055,7 +1055,7 @@ public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.i
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map48.size);
           for (int _i49 = 0; _i49 < _map48.size; ++_i49)
           {
-            ByteBuffer _key50; // optional
+            ByteBuffer _key50; // required
             ByteBuffer _val51; // required
             _key50 = iprot.readBinary();
             _val51 = iprot.readBinary();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
index 0486b6b..746a657 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
@@ -49,6 +49,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)9);
   private static final org.apache.thrift.protocol.TField AUTHORIZATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizations", org.apache.thrift.protocol.TType.STRUCT, (short)10);
   private static final org.apache.thrift.protocol.TField REVERSED_FIELD_DESC = new org.apache.thrift.protocol.TField("reversed", org.apache.thrift.protocol.TType.BOOL, (short)11);
+  private static final org.apache.thrift.protocol.TField CACHE_BLOCKS_FIELD_DESC = new org.apache.thrift.protocol.TField("cacheBlocks", org.apache.thrift.protocol.TType.BOOL, (short)12);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -67,6 +68,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   public Map<ByteBuffer,ByteBuffer> attributes; // optional
   public TAuthorization authorizations; // optional
   public boolean reversed; // optional
+  public boolean cacheBlocks; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -80,7 +82,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     BATCH_SIZE((short)8, "batchSize"),
     ATTRIBUTES((short)9, "attributes"),
     AUTHORIZATIONS((short)10, "authorizations"),
-    REVERSED((short)11, "reversed");
+    REVERSED((short)11, "reversed"),
+    CACHE_BLOCKS((short)12, "cacheBlocks");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -117,6 +120,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
           return AUTHORIZATIONS;
         case 11: // REVERSED
           return REVERSED;
+        case 12: // CACHE_BLOCKS
+          return CACHE_BLOCKS;
         default:
           return null;
       }
@@ -161,8 +166,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
   private static final int __MAXVERSIONS_ISSET_ID = 1;
   private static final int __BATCHSIZE_ISSET_ID = 2;
   private static final int __REVERSED_ISSET_ID = 3;
+  private static final int __CACHEBLOCKS_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.COLUMNS,_Fields.CACHING,_Fields.MAX_VERSIONS,_Fields.TIME_RANGE,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.REVERSED};
+  private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.COLUMNS,_Fields.CACHING,_Fields.MAX_VERSIONS,_Fields.TIME_RANGE,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.REVERSED,_Fields.CACHE_BLOCKS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -191,6 +197,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TAuthorization.class)));
     tmpMap.put(_Fields.REVERSED, new org.apache.thrift.meta_data.FieldMetaData("reversed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CACHE_BLOCKS, new org.apache.thrift.meta_data.FieldMetaData("cacheBlocks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TScan.class, metaDataMap);
   }
@@ -251,6 +259,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       this.authorizations = new TAuthorization(other.authorizations);
     }
     this.reversed = other.reversed;
+    this.cacheBlocks = other.cacheBlocks;
   }
 
   public TScan deepCopy() {
@@ -274,6 +283,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     this.authorizations = null;
     setReversedIsSet(false);
     this.reversed = false;
+    setCacheBlocksIsSet(false);
+    this.cacheBlocks = false;
   }
 
   public byte[] getStartRow() {
@@ -592,6 +603,29 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REVERSED_ISSET_ID, value);
   }
 
+  public boolean isCacheBlocks() {
+    return this.cacheBlocks;
+  }
+
+  public TScan setCacheBlocks(boolean cacheBlocks) {
+    this.cacheBlocks = cacheBlocks;
+    setCacheBlocksIsSet(true);
+    return this;
+  }
+
+  public void unsetCacheBlocks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  /** Returns true if field cacheBlocks is set (has been assigned a value) and false otherwise */
+  public boolean isSetCacheBlocks() {
+    return EncodingUtils.testBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID);
+  }
+
+  public void setCacheBlocksIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHEBLOCKS_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case START_ROW:
@@ -682,6 +716,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       }
       break;
 
+    case CACHE_BLOCKS:
+      if (value == null) {
+        unsetCacheBlocks();
+      } else {
+        setCacheBlocks((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -720,6 +762,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     case REVERSED:
       return Boolean.valueOf(isReversed());
 
+    case CACHE_BLOCKS:
+      return Boolean.valueOf(isCacheBlocks());
+
     }
     throw new IllegalStateException();
   }
@@ -753,6 +798,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       return isSetAuthorizations();
     case REVERSED:
       return isSetReversed();
+    case CACHE_BLOCKS:
+      return isSetCacheBlocks();
     }
     throw new IllegalStateException();
   }
@@ -869,6 +916,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         return false;
     }
 
+    boolean this_present_cacheBlocks = true && this.isSetCacheBlocks();
+    boolean that_present_cacheBlocks = true && that.isSetCacheBlocks();
+    if (this_present_cacheBlocks || that_present_cacheBlocks) {
+      if (!(this_present_cacheBlocks && that_present_cacheBlocks))
+        return false;
+      if (this.cacheBlocks != that.cacheBlocks)
+        return false;
+    }
+
     return true;
   }
 
@@ -931,6 +987,11 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
     if (present_reversed)
       builder.append(reversed);
 
+    boolean present_cacheBlocks = true && (isSetCacheBlocks());
+    builder.append(present_cacheBlocks);
+    if (present_cacheBlocks)
+      builder.append(cacheBlocks);
+
     return builder.toHashCode();
   }
 
@@ -1052,6 +1113,16 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCacheBlocks()).compareTo(typedOther.isSetCacheBlocks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCacheBlocks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cacheBlocks, typedOther.cacheBlocks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1165,6 +1236,12 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       sb.append(this.reversed);
       first = false;
     }
+    if (isSetCacheBlocks()) {
+      if (!first) sb.append(", ");
+      sb.append("cacheBlocks:");
+      sb.append(this.cacheBlocks);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1299,7 +1376,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
                 struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map109.size);
                 for (int _i110 = 0; _i110 < _map109.size; ++_i110)
                 {
-                  ByteBuffer _key111; // optional
+                  ByteBuffer _key111; // required
                   ByteBuffer _val112; // required
                   _key111 = iprot.readBinary();
                   _val112 = iprot.readBinary();
@@ -1329,6 +1406,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 12: // CACHE_BLOCKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.cacheBlocks = iprot.readBool();
+              struct.setCacheBlocksIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1428,6 +1513,11 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         oprot.writeBool(struct.reversed);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeFieldBegin(CACHE_BLOCKS_FIELD_DESC);
+        oprot.writeBool(struct.cacheBlocks);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1479,7 +1569,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       if (struct.isSetReversed()) {
         optionals.set(10);
       }
-      oprot.writeBitSet(optionals, 11);
+      if (struct.isSetCacheBlocks()) {
+        optionals.set(11);
+      }
+      oprot.writeBitSet(optionals, 12);
       if (struct.isSetStartRow()) {
         oprot.writeBinary(struct.startRow);
       }
@@ -1526,12 +1619,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
       if (struct.isSetReversed()) {
         oprot.writeBool(struct.reversed);
       }
+      if (struct.isSetCacheBlocks()) {
+        oprot.writeBool(struct.cacheBlocks);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, TScan struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(11);
+      BitSet incoming = iprot.readBitSet(12);
       if (incoming.get(0)) {
         struct.startRow = iprot.readBinary();
         struct.setStartRowIsSet(true);
@@ -1581,7 +1677,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
           struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map120.size);
           for (int _i121 = 0; _i121 < _map120.size; ++_i121)
           {
-            ByteBuffer _key122; // optional
+            ByteBuffer _key122; // required
             ByteBuffer _val123; // required
             _key122 = iprot.readBinary();
             _val123 = iprot.readBinary();
@@ -1599,6 +1695,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
         struct.reversed = iprot.readBool();
         struct.setReversedIsSet(true);
       }
+      if (incoming.get(11)) {
+        struct.cacheBlocks = iprot.readBool();
+        struct.setCacheBlocksIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
index f2d264a..0669ef8 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
@@ -150,7 +150,8 @@ struct TScan {
   6:optional Text filterString,
   7:optional i32 batchSize,
   8:optional bool sortColumns,
-  9:optional bool reversed 
+  9:optional bool reversed,
+  10:optional bool cacheBlocks
 }
 
 //

http://git-wip-us.apache.org/repos/asf/hbase/blob/4678c5d9/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index 7c6d6a2..07d71f8 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -224,6 +224,7 @@ struct TScan {
   9: optional map<binary, binary> attributes
   10: optional TAuthorization authorizations
   11: optional bool reversed
+  12: optional bool cacheBlocks
 }
 
 /**