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

[01/43] hbase git commit: HBASE-14995 Optimize setting tagsPresent in DefaultMemStore.java (huaxiang sun)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 3dec8582f -> 5eefe1317


HBASE-14995 Optimize setting tagsPresent in DefaultMemStore.java (huaxiang sun)


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

Branch: refs/heads/hbase-12439
Commit: 546ac1cbb379acb198b2debabbf14c878ffb8a33
Parents: 3dec858
Author: tedyu <yu...@gmail.com>
Authored: Wed Dec 16 18:16:38 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Dec 16 18:16:38 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/546ac1cb/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 89ae0d1..14c2e32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -240,7 +240,7 @@ public class DefaultMemStore implements MemStore {
     // When we use ACL CP or Visibility CP which deals with Tags during
     // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not
     // parse the byte[] to identify the tags length.
-    if(e.getTagsLength() > 0) {
+    if (!tagsPresent && (e.getTagsLength() > 0)) {
       tagsPresent = true;
     }
     setOldestEditTimeToNow();


[03/43] hbase git commit: HBASE-14999 Remove ref to org.mortbay.log.Log.

Posted by sy...@apache.org.
HBASE-14999 Remove ref to org.mortbay.log.Log.


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

Branch: refs/heads/hbase-12439
Commit: cf458d302385f8e32dd444f4820cfd72a8694862
Parents: d78eddf
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Dec 17 18:01:16 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Dec 17 18:01:16 2015 +0530

----------------------------------------------------------------------
 .../hbase/client/FlushRegionCallable.java       |  7 ++++--
 .../apache/hadoop/hbase/TestClassFinder.java    | 18 ++++++++------
 .../hbase/client/ClientSideRegionScanner.java   |  9 ++++---
 .../hbase/client/TestHBaseAdminNoCluster.java   | 14 +++++++----
 .../io/encoding/TestDataBlockEncoders.java      | 23 ++++++++++-------
 .../regionserver/TestHRegionOnCluster.java      | 26 ++++++++++++--------
 .../regionserver/TestRegionServerNoMaster.java  |  6 +++--
 .../regionserver/TestSplitWalDataLoss.java      |  7 ++++--
 8 files changed, 70 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
index b2c4a57..73bdb74 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FlushRegionCallable.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.mortbay.log.Log;
 
 import com.google.protobuf.ServiceException;
 
@@ -41,6 +42,8 @@ import com.google.protobuf.ServiceException;
 @InterfaceAudience.Private
 public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionResponse> {
 
+  private static final Log LOG = LogFactory.getLog(FlushRegionCallable.class);
+
   private final byte[] regionName;
   private final boolean writeFlushWalMarker;
   private boolean reload;
@@ -78,7 +81,7 @@ public class FlushRegionCallable extends RegionAdminServiceCallable<FlushRegionR
       if (!reload) {
         throw new IOException("Cached location seems to be different than requested region.");
       }
-      Log.info("Skipping flush region, because the located region "
+      LOG.info("Skipping flush region, because the located region "
           + Bytes.toStringBinary(location.getRegionInfo().getRegionName()) + " is different than "
           + " requested region " + Bytes.toStringBinary(regionName));
       return FlushRegionResponse.newBuilder()

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java
index 0b83d05..5154810 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestClassFinder.java
@@ -42,6 +42,8 @@ import java.util.jar.Manifest;
 import javax.tools.JavaCompiler;
 import javax.tools.ToolProvider;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.AfterClass;
@@ -50,10 +52,12 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.mortbay.log.Log;
 
 @Category({MiscTests.class, SmallTests.class})
 public class TestClassFinder {
+
+  private static final Log LOG = LogFactory.getLog(TestClassFinder.class);
+
   @Rule public TestName name = new TestName();
   private static final HBaseCommonTestingUtility testUtil = new HBaseCommonTestingUtility();
   private static final String BASEPKG = "tfcpkg";
@@ -79,7 +83,7 @@ public class TestClassFinder {
       deleteTestDir();
     }
     assertTrue(testDir.mkdirs());
-    Log.info("Using new, clean directory=" + testDir);
+    LOG.info("Using new, clean directory=" + testDir);
   }
 
   @AfterClass
@@ -142,7 +146,7 @@ public class TestClassFinder {
   public void testClassFinderFiltersByNameInJar() throws Exception {
     final long counter = testCounter.incrementAndGet();
     final String classNamePrefix = name.getMethodName();
-    Log.info("Created jar " + createAndLoadJar("", classNamePrefix, counter));
+    LOG.info("Created jar " + createAndLoadJar("", classNamePrefix, counter));
 
     ClassFinder.FileNameFilter notExcNameFilter = new ClassFinder.FileNameFilter() {
       @Override
@@ -162,7 +166,7 @@ public class TestClassFinder {
   public void testClassFinderFiltersByClassInJar() throws Exception {
     final long counter = testCounter.incrementAndGet();
     final String classNamePrefix = name.getMethodName();
-    Log.info("Created jar " + createAndLoadJar("", classNamePrefix, counter));
+    LOG.info("Created jar " + createAndLoadJar("", classNamePrefix, counter));
 
     final ClassFinder.ClassFilter notExcClassFilter = new ClassFinder.ClassFilter() {
       @Override
@@ -224,7 +228,7 @@ public class TestClassFinder {
     final long counter = testCounter.incrementAndGet();
     final String classNamePrefix = name.getMethodName();
     String pkgNameSuffix = name.getMethodName();
-    Log.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
+    LOG.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
     ClassFinder allClassesFinder = new ClassFinder();
     String pkgName = makePackageName(pkgNameSuffix, counter);
     Set<Class<?>> allClasses = allClassesFinder.findClasses(pkgName, false);
@@ -247,7 +251,7 @@ public class TestClassFinder {
     final long counter = testCounter.incrementAndGet();
     final String classNamePrefix = name.getMethodName();
     String pkgNameSuffix = name.getMethodName();
-    Log.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
+    LOG.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
     final String classNameToFilterOut = classNamePrefix + counter;
     final ClassFinder.FileNameFilter notThisFilter = new ClassFinder.FileNameFilter() {
       @Override
@@ -272,7 +276,7 @@ public class TestClassFinder {
     final long counter = testCounter.incrementAndGet();
     final String classNamePrefix = name.getMethodName();
     String pkgNameSuffix = name.getMethodName();
-    Log.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
+    LOG.info("Created jar " + createAndLoadJar(pkgNameSuffix, classNamePrefix, counter));
     final Class<?> clazz = makeClass(pkgNameSuffix, classNamePrefix, counter);
     final ClassFinder.ClassFilter notThisFilter = new ClassFinder.ClassFilter() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index 1ec085f..dde2f10 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -33,7 +35,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.mortbay.log.Log;
 
 /**
  * A client scanner for a region opened for read-only on the client side. Assumes region data
@@ -42,6 +43,8 @@ import org.mortbay.log.Log;
 @InterfaceAudience.Private
 public class ClientSideRegionScanner extends AbstractClientScanner {
 
+  private static final Log LOG = LogFactory.getLog(ClientSideRegionScanner.class);
+
   private HRegion region;
   RegionScanner scanner;
   List<Cell> values;
@@ -96,7 +99,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
         this.scanner.close();
         this.scanner = null;
       } catch (IOException ex) {
-        Log.warn("Exception while closing scanner", ex);
+        LOG.warn("Exception while closing scanner", ex);
       }
     }
     if (this.region != null) {
@@ -105,7 +108,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
         this.region.close(true);
         this.region = null;
       } catch (IOException ex) {
-        Log.warn("Exception while closing region", ex);
+        LOG.warn("Exception while closing region", ex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index 4b8de7a..8b2b733 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -24,9 +24,9 @@ import static org.mockito.Mockito.when;
 import java.io.IOException;
 import java.util.ArrayList;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -46,6 +46,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -53,13 +55,15 @@ import org.mockito.Matchers;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-import org.mortbay.log.Log;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 
 @Category({SmallTests.class, ClientTests.class})
 public class TestHBaseAdminNoCluster {
+
+  private static final Log LOG = LogFactory.getLog(TestHBaseAdminNoCluster.class);
+
   /**
    * Verify that PleaseHoldException gets retried.
    * HBASE-8764
@@ -99,7 +103,7 @@ public class TestHBaseAdminNoCluster {
         admin.createTable(htd, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
         fail();
       } catch (RetriesExhaustedException e) {
-        Log.info("Expected fail", e);
+        LOG.info("Expected fail", e);
       }
       // Assert we were called 'count' times.
       Mockito.verify(masterAdmin, Mockito.atLeast(count)).createTable((RpcController)Mockito.any(),
@@ -317,7 +321,7 @@ public class TestHBaseAdminNoCluster {
         caller.call(admin); // invoke the HBaseAdmin method
         fail();
       } catch (RetriesExhaustedException e) {
-        Log.info("Expected fail", e);
+        LOG.info("Expected fail", e);
       }
       // Assert we were called 'count' times.
       caller.verify(masterAdmin, count);

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
index 64c74e8..ce48ca1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
@@ -31,12 +31,14 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -58,7 +60,7 @@ import org.junit.rules.TestRule;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
-import org.mortbay.log.Log;
+
 
 /**
  * Test all of the data block encoding algorithms for correctness. Most of the
@@ -67,6 +69,9 @@ import org.mortbay.log.Log;
 @Category({IOTests.class, LargeTests.class})
 @RunWith(Parameterized.class)
 public class TestDataBlockEncoders {
+
+  private static final Log LOG = LogFactory.getLog(TestDataBlockEncoders.class);
+
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
       withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
@@ -189,7 +194,7 @@ public class TestDataBlockEncoders {
     List<DataBlockEncoder.EncodedSeeker> encodedSeekers = 
         new ArrayList<DataBlockEncoder.EncodedSeeker>();
     for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
-      Log.info("Encoding: " + encoding);
+      LOG.info("Encoding: " + encoding);
       // Off heap block data support not added for PREFIX_TREE DBE yet.
       // TODO remove this once support is added. HBASE-12298
       if (this.useOffheapData && encoding == DataBlockEncoding.PREFIX_TREE) continue;
@@ -197,7 +202,7 @@ public class TestDataBlockEncoders {
       if (encoder == null) {
         continue;
       }
-      Log.info("Encoder: " + encoder);
+      LOG.info("Encoder: " + encoder);
       ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
           getEncodingContext(Compression.Algorithm.NONE, encoding), this.useOffheapData);
       HFileContext meta = new HFileContextBuilder()
@@ -211,7 +216,7 @@ public class TestDataBlockEncoders {
       seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
       encodedSeekers.add(seeker);
     }
-    Log.info("Testing it!");
+    LOG.info("Testing it!");
     // test it!
     // try a few random seeks
     for (boolean seekBefore : new boolean[] { false, true }) {
@@ -229,7 +234,7 @@ public class TestDataBlockEncoders {
     }
 
     // check edge cases
-    Log.info("Checking edge cases");
+    LOG.info("Checking edge cases");
     checkSeekingConsistency(encodedSeekers, false, sampleKv.get(0));
     for (boolean seekBefore : new boolean[] { false, true }) {
       checkSeekingConsistency(encodedSeekers, seekBefore, sampleKv.get(sampleKv.size() - 1));
@@ -237,7 +242,7 @@ public class TestDataBlockEncoders {
       Cell lastMidKv =CellUtil.createLastOnRowCol(midKv);
       checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
     }
-    Log.info("Done");
+    LOG.info("Done");
   }
 
   static ByteBuffer encodeKeyValues(DataBlockEncoding encoding, List<KeyValue> kvs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
index 98d98aa..108af10 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
@@ -17,12 +17,16 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -42,7 +46,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mortbay.log.Log;
+
 
 /**
  * Tests that need to spin up a cluster testing an {@link HRegion}.  Use
@@ -51,6 +55,8 @@ import org.mortbay.log.Log;
  */
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestHRegionOnCluster {
+
+  private static final Log LOG = LogFactory.getLog(TestHRegionOnCluster.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @Test (timeout=300000)
@@ -75,7 +81,7 @@ public class TestHRegionOnCluster {
       assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
 
       // Put data: r1->v1
-      Log.info("Loading r1 to v1 into " + TABLENAME);
+      LOG.info("Loading r1 to v1 into " + TABLENAME);
       Table table = TEST_UTIL.getConnection().getTable(TABLENAME);
       putDataAndVerify(table, "r1", FAMILY, "v1", 1);
 
@@ -94,7 +100,7 @@ public class TestHRegionOnCluster {
       assertFalse(originServer.equals(targetServer));
 
       TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
-      Log.info("Moving " + regionInfo.getEncodedName() + " to " + targetServer.getServerName());
+      LOG.info("Moving " + regionInfo.getEncodedName() + " to " + targetServer.getServerName());
       hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(),
           Bytes.toBytes(targetServer.getServerName().getServerName()));
       do {
@@ -102,12 +108,12 @@ public class TestHRegionOnCluster {
       } while (cluster.getServerWith(regionInfo.getRegionName()) == originServerNum);
 
       // Put data: r2->v2
-      Log.info("Loading r2 to v2 into " + TABLENAME);
+      LOG.info("Loading r2 to v2 into " + TABLENAME);
       putDataAndVerify(table, "r2", FAMILY, "v2", 2);
 
       TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
       // Move region to origin server
-      Log.info("Moving " + regionInfo.getEncodedName() + " to " + originServer.getServerName());
+      LOG.info("Moving " + regionInfo.getEncodedName() + " to " + originServer.getServerName());
       hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(),
           Bytes.toBytes(originServer.getServerName().getServerName()));
       do {
@@ -115,11 +121,11 @@ public class TestHRegionOnCluster {
       } while (cluster.getServerWith(regionInfo.getRegionName()) == targetServerNum);
 
       // Put data: r3->v3
-      Log.info("Loading r3 to v3 into " + TABLENAME);
+      LOG.info("Loading r3 to v3 into " + TABLENAME);
       putDataAndVerify(table, "r3", FAMILY, "v3", 3);
 
       // Kill target server
-      Log.info("Killing target server " + targetServer.getServerName());
+      LOG.info("Killing target server " + targetServer.getServerName());
       targetServer.kill();
       cluster.getRegionServerThreads().get(targetServerNum).join();
       // Wait until finish processing of shutdown
@@ -127,12 +133,12 @@ public class TestHRegionOnCluster {
         Thread.sleep(5);
       }
       // Kill origin server
-      Log.info("Killing origin server " + targetServer.getServerName());
+      LOG.info("Killing origin server " + targetServer.getServerName());
       originServer.kill();
       cluster.getRegionServerThreads().get(originServerNum).join();
 
       // Put data: r4->v4
-      Log.info("Loading r4 to v4 into " + TABLENAME);
+      LOG.info("Loading r4 to v4 into " + TABLENAME);
       putDataAndVerify(table, "r4", FAMILY, "v4", 4);
 
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index e8a2134..03b43f1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -48,7 +50,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mortbay.log.Log;
 
 import com.google.protobuf.ServiceException;
 
@@ -58,6 +59,7 @@ import com.google.protobuf.ServiceException;
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestRegionServerNoMaster {
 
+  private static final Log LOG = LogFactory.getLog(TestRegionServerNoMaster.class);
   private static final int NB_SERVERS = 1;
   private static Table table;
   private static final byte[] row = "ee".getBytes();
@@ -95,7 +97,7 @@ public class TestRegionServerNoMaster {
     ServerName masterAddr = master.getServerName();
     master.stopMaster();
 
-    Log.info("Waiting until master thread exits");
+    LOG.info("Waiting until master thread exits");
     while (masterThread != null && masterThread.isAlive()) {
       Threads.sleep(100);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cf458d30/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
index 870a3bf..1294266 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitWalDataLoss.java
@@ -26,6 +26,8 @@ import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.commons.lang.mutable.MutableBoolean;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -53,7 +55,6 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Matchers;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-import org.mortbay.log.Log;
 
 /**
  * Testcase for https://issues.apache.org/jira/browse/HBASE-13811
@@ -61,6 +62,8 @@ import org.mortbay.log.Log;
 @Category({ MediumTests.class })
 public class TestSplitWalDataLoss {
 
+  private static final Log LOG = LogFactory.getLog(TestSplitWalDataLoss.class);
+
   private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
 
   private NamespaceDescriptor namespace = NamespaceDescriptor.create(getClass().getSimpleName())
@@ -122,7 +125,7 @@ public class TestSplitWalDataLoss {
               .addColumn(family, qualifier, Bytes.toBytes("val0")));
     }
     long oldestSeqIdOfStore = region.getOldestSeqIdOfStore(family);
-    Log.info("CHANGE OLDEST " + oldestSeqIdOfStore);
+    LOG.info("CHANGE OLDEST " + oldestSeqIdOfStore);
     assertTrue(oldestSeqIdOfStore > HConstants.NO_SEQNUM);
     rs.cacheFlusher.requestFlush(spiedRegion, false);
     synchronized (flushed) {


[02/43] hbase git commit: HBASE-14995 Revert according to Ram's feedback

Posted by sy...@apache.org.
HBASE-14995 Revert according to Ram's feedback


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

Branch: refs/heads/hbase-12439
Commit: d78eddfdc8bad5068600e28a039276cc55063ce2
Parents: 546ac1c
Author: tedyu <yu...@gmail.com>
Authored: Wed Dec 16 21:54:48 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Dec 16 21:54:48 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/d78eddfd/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
index 14c2e32..89ae0d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java
@@ -240,7 +240,7 @@ public class DefaultMemStore implements MemStore {
     // When we use ACL CP or Visibility CP which deals with Tags during
     // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not
     // parse the byte[] to identify the tags length.
-    if (!tagsPresent && (e.getTagsLength() > 0)) {
+    if(e.getTagsLength() > 0) {
       tagsPresent = true;
     }
     setOldestEditTimeToNow();


[13/43] hbase git commit: HBASE-15003 remove BoundedConcurrentLinkedQueue.

Posted by sy...@apache.org.
HBASE-15003 remove BoundedConcurrentLinkedQueue.

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/hbase-12439
Commit: ba04e0372da560715bb4b1aa329c0091764340ea
Parents: 58342a8
Author: Sean Busbey <bu...@cloudera.com>
Authored: Thu Dec 17 22:50:19 2015 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Dec 18 12:26:38 2015 -0600

----------------------------------------------------------------------
 .../util/BoundedConcurrentLinkedQueue.java      | 122 --------------
 .../util/TestBoundedConcurrentLinkedQueue.java  | 161 -------------------
 2 files changed, 283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ba04e037/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedConcurrentLinkedQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedConcurrentLinkedQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedConcurrentLinkedQueue.java
deleted file mode 100644
index f66771b..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BoundedConcurrentLinkedQueue.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.util.Collection;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * A ConcurrentLinkedQueue that enforces a maximum queue size.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class BoundedConcurrentLinkedQueue<T> extends ConcurrentLinkedQueue<T> {
-  private static final long serialVersionUID = 1L;
-  private final AtomicLong size = new AtomicLong(0L);
-  private final long maxSize;
-
-  public BoundedConcurrentLinkedQueue() {
-    this(Long.MAX_VALUE);
-  }
-
-  public BoundedConcurrentLinkedQueue(long maxSize) {
-    super();
-    this.maxSize = maxSize;
-  }
-
-  @Override
-  public boolean addAll(Collection<? extends T> c) {
-    for (;;) {
-      long currentSize = size.get();
-      long nextSize = currentSize + c.size();
-      if (nextSize > maxSize) { // already exceeded limit
-        return false;
-      }
-      if (size.compareAndSet(currentSize, nextSize)) {
-        break;
-      }
-    }
-    return super.addAll(c); // Always true for ConcurrentLinkedQueue
-  }
-
-  @Override
-  public void clear() {
-    // override this method to batch update size.
-    long removed = 0L;
-    while (super.poll() != null) {
-      removed++;
-    }
-    size.addAndGet(-removed);
-  }
-
-  @Override
-  public boolean offer(T e) {
-    for (;;) {
-      long currentSize = size.get();
-      if (currentSize >= maxSize) { // already exceeded limit
-        return false;
-      }
-      if (size.compareAndSet(currentSize, currentSize + 1)) {
-        break;
-      }
-    }
-    return super.offer(e); // Always true for ConcurrentLinkedQueue
-  }
-
-  @Override
-  public T poll() {
-    T result = super.poll();
-    if (result != null) {
-      size.decrementAndGet();
-    }
-    return result;
-  }
-
-  @Override
-  public boolean remove(Object o) {
-    boolean result = super.remove(o);
-    if (result) {
-      size.decrementAndGet();
-    }
-    return result;
-  }
-
-  @Override
-  public int size() {
-    return (int) size.get();
-  }
-
-  public void drainTo(Collection<T> list) {
-    long removed = 0;
-    for (T element; (element = super.poll()) != null;) {
-      list.add(element);
-      removed++;
-    }
-    // Limit the number of operations on size by only reporting size change after the drain is
-    // completed.
-    size.addAndGet(-removed);
-  }
-
-  public long remainingCapacity() {
-    return maxSize - size.get();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ba04e037/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedConcurrentLinkedQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedConcurrentLinkedQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedConcurrentLinkedQueue.java
deleted file mode 100644
index f6e6ac5..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestBoundedConcurrentLinkedQueue.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hadoop.hbase.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MiscTests.class, SmallTests.class })
-public class TestBoundedConcurrentLinkedQueue {
-  private final static int CAPACITY = 16;
-
-  private BoundedConcurrentLinkedQueue<Long> queue;
-
-  @Before
-  public void setUp() throws Exception {
-    this.queue = new BoundedConcurrentLinkedQueue<Long>(CAPACITY);
-  }
-
-  @Test
-  public void testOfferAndPoll() throws Exception {
-    // Offer
-    for (long i = 1; i <= CAPACITY; ++i) {
-      assertTrue(queue.offer(i));
-      assertEquals(i, queue.size());
-      assertEquals(CAPACITY - i, queue.remainingCapacity());
-    }
-    assertFalse(queue.offer(0L));
-
-    // Poll
-    for (int i = 1; i <= CAPACITY; ++i) {
-      long l = queue.poll();
-      assertEquals(i, l);
-      assertEquals(CAPACITY - i, queue.size());
-      assertEquals(i, queue.remainingCapacity());
-    }
-    assertEquals(null, queue.poll());
-  }
-
-  @Test
-  public void testDrain() throws Exception {
-    // Offer
-    for (long i = 1; i <= CAPACITY; ++i) {
-      assertTrue(queue.offer(i));
-      assertEquals(i, queue.size());
-      assertEquals(CAPACITY - i, queue.remainingCapacity());
-    }
-    assertFalse(queue.offer(0L));
-
-    // Drain
-    List<Long> list = new ArrayList<Long>();
-    queue.drainTo(list);
-    assertEquals(null, queue.poll());
-    assertEquals(0, queue.size());
-    assertEquals(CAPACITY, queue.remainingCapacity());
-  }
-
-  @Test
-  public void testClear() {
-    // Offer
-    for (long i = 1; i <= CAPACITY; ++i) {
-      assertTrue(queue.offer(i));
-      assertEquals(i, queue.size());
-      assertEquals(CAPACITY - i, queue.remainingCapacity());
-    }
-    assertFalse(queue.offer(0L));
-
-    queue.clear();
-    assertEquals(null, queue.poll());
-    assertEquals(0, queue.size());
-    assertEquals(CAPACITY, queue.remainingCapacity());
-  }
-
-  @Test
-  public void testMultiThread() throws InterruptedException {
-    int offerThreadCount = 10;
-    int pollThreadCount = 5;
-    int duration = 5000; // ms
-    final AtomicBoolean stop = new AtomicBoolean(false);
-    Thread[] offerThreads = new Thread[offerThreadCount];
-    for (int i = 0; i < offerThreadCount; i++) {
-      offerThreads[i] = new Thread("offer-thread-" + i) {
-
-        @Override
-        public void run() {
-          Random rand = new Random();
-          while (!stop.get()) {
-            queue.offer(rand.nextLong());
-            try {
-              Thread.sleep(1);
-            } catch (InterruptedException e) {
-            }
-          }
-        }
-
-      };
-    }
-    Thread[] pollThreads = new Thread[pollThreadCount];
-    for (int i = 0; i < pollThreadCount; i++) {
-      pollThreads[i] = new Thread("poll-thread-" + i) {
-
-        @Override
-        public void run() {
-          while (!stop.get()) {
-            queue.poll();
-            try {
-              Thread.sleep(1);
-            } catch (InterruptedException e) {
-            }
-          }
-        }
-
-      };
-    }
-    for (Thread t : offerThreads) {
-      t.start();
-    }
-    for (Thread t : pollThreads) {
-      t.start();
-    }
-    long startTime = System.currentTimeMillis();
-    while (System.currentTimeMillis() - startTime < duration) {
-      assertTrue(queue.size() <= CAPACITY);
-      Thread.yield();
-    }
-    stop.set(true);
-    for (Thread t : offerThreads) {
-      t.join();
-    }
-    for (Thread t : pollThreads) {
-      t.join();
-    }
-    assertTrue(queue.size() <= CAPACITY);
-  }
-}


[14/43] hbase git commit: HBASE-15001 Fix thread-safety issues with replication

Posted by sy...@apache.org.
HBASE-15001 Fix thread-safety issues with replication

ReplicationSinkManager and HBaseInterClusterReplicationEndpoint
perform certain unsafe operations which might lead to undesirable
behavior with multiwal enabled.

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


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

Branch: refs/heads/hbase-12439
Commit: 48113d7572bbea2f05ae619882d43aa41827f147
Parents: ba04e03
Author: Ashu Pachauri <as...@gmail.com>
Authored: Thu Dec 17 13:25:39 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Dec 18 11:48:55 2015 -0800

----------------------------------------------------------------------
 .../HBaseInterClusterReplicationEndpoint.java   | 27 +++++++++++++++-----
 .../regionserver/ReplicationSinkManager.java    | 21 ++++++++++-----
 .../TestReplicationSinkManager.java             | 26 +++++++++----------
 3 files changed, 49 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/48113d75/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 70cc420..78e3e00 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -143,9 +143,9 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     int sleepMultiplier = 1;
 
     // Connect to peer cluster first, unless we have to stop
-    while (this.isRunning() && replicationSinkMgr.getSinks().size() == 0) {
+    while (this.isRunning() && replicationSinkMgr.getNumSinks() == 0) {
       replicationSinkMgr.chooseSinks();
-      if (this.isRunning() && replicationSinkMgr.getSinks().size() == 0) {
+      if (this.isRunning() && replicationSinkMgr.getNumSinks() == 0) {
         if (sleepForRetries("Waiting for peers", sleepMultiplier)) {
           sleepMultiplier++;
         }
@@ -180,19 +180,24 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
     List<Entry> entries = replicateContext.getEntries();
     String walGroupId = replicateContext.getWalGroupId();
     int sleepMultiplier = 1;
+    int numReplicated = 0;
 
     if (!peersSelected && this.isRunning()) {
       connectToPeers();
       peersSelected = true;
     }
 
-    if (replicationSinkMgr.getSinks().size() == 0) {
+    int numSinks = replicationSinkMgr.getNumSinks();
+    if (numSinks == 0) {
+      LOG.warn("No replication sinks found, returning without replicating. The source should retry"
+          + " with the same set of edits.");
       return false;
     }
+
     // minimum of: configured threads, number of 100-waledit batches,
     //  and number of current sinks
-    int n = Math.min(Math.min(this.maxThreads, entries.size()/100+1),
-      replicationSinkMgr.getSinks().size());
+    int n = Math.min(Math.min(this.maxThreads, entries.size()/100+1), numSinks);
+
     List<List<Entry>> entryLists = new ArrayList<List<Entry>>(n);
     if (n == 1) {
       entryLists.add(entries);
@@ -237,7 +242,11 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
             // wait for all futures, remove successful parts
             // (only the remaining parts will be retried)
             Future<Integer> f = pool.take();
-            entryLists.set(f.get().intValue(), Collections.<Entry>emptyList());
+            int index = f.get().intValue();
+            int batchSize =  entryLists.get(index).size();
+            entryLists.set(index, Collections.<Entry>emptyList());
+            // Now, we have marked the batch as done replicating, record its size
+            numReplicated += batchSize;
           } catch (InterruptedException ie) {
             iox =  new IOException(ie);
           } catch (ExecutionException ee) {
@@ -249,6 +258,12 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
           // if we had any exceptions, try again
           throw iox;
         }
+        if (numReplicated != entries.size()) {
+          // Something went wrong here and we don't know what, let's just fail and retry.
+          LOG.warn("The number of edits replicated is different from the number received,"
+              + " failing for now.");
+          return false;
+        }
         // update metrics
         this.metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
           walGroupId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/48113d75/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
index 76fa6c2..0469f9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSinkManager.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -105,7 +106,7 @@ public class ReplicationSinkManager {
    *
    * @return a replication sink to replicate to
    */
-  public SinkPeer getReplicationSink() throws IOException {
+  public synchronized SinkPeer getReplicationSink() throws IOException {
     if (endpoint.getLastRegionServerUpdate() > this.lastUpdateToPeers || sinks.isEmpty()) {
       LOG.info("Current list of sinks is out of date or empty, updating");
       chooseSinks();
@@ -127,7 +128,7 @@ public class ReplicationSinkManager {
    * @param sinkPeer
    *          The SinkPeer that had a failed replication attempt on it
    */
-  public void reportBadSink(SinkPeer sinkPeer) {
+  public synchronized void reportBadSink(SinkPeer sinkPeer) {
     ServerName serverName = sinkPeer.getServerName();
     int badReportCount = (badReportCounts.containsKey(serverName)
                     ? badReportCounts.get(serverName) : 0) + 1;
@@ -146,11 +147,14 @@ public class ReplicationSinkManager {
    * @param sinkPeer
    *          The SinkPeer that had a failed replication attempt on it
    */
-  public void reportSinkSuccess(SinkPeer sinkPeer) {
+  public synchronized void reportSinkSuccess(SinkPeer sinkPeer) {
     badReportCounts.remove(sinkPeer.getServerName());
   }
 
-  void chooseSinks() {
+  /**
+   * Refresh the list of sinks.
+   */
+  public synchronized void chooseSinks() {
     List<ServerName> slaveAddresses = endpoint.getRegionServers();
     Collections.shuffle(slaveAddresses, random);
     int numSinks = (int) Math.ceil(slaveAddresses.size() * ratio);
@@ -159,8 +163,13 @@ public class ReplicationSinkManager {
     badReportCounts.clear();
   }
 
-  List<ServerName> getSinks() {
-    return sinks;
+  public synchronized int getNumSinks() {
+    return sinks.size();
+  }
+
+  @VisibleForTesting
+  protected List<ServerName> getSinksForTesting() {
+    return Collections.unmodifiableList(sinks);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/48113d75/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
index 57c3196..104753a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSinkManager.java
@@ -67,7 +67,7 @@ public class TestReplicationSinkManager {
 
     sinkManager.chooseSinks();
 
-    assertEquals(2, sinkManager.getSinks().size());
+    assertEquals(2, sinkManager.getNumSinks());
 
   }
 
@@ -81,7 +81,7 @@ public class TestReplicationSinkManager {
 
     sinkManager.chooseSinks();
 
-    assertEquals(1, sinkManager.getSinks().size());
+    assertEquals(1, sinkManager.getNumSinks());
   }
 
   @Test
@@ -93,14 +93,14 @@ public class TestReplicationSinkManager {
 
     sinkManager.chooseSinks();
     // Sanity check
-    assertEquals(1, sinkManager.getSinks().size());
+    assertEquals(1, sinkManager.getNumSinks());
 
     SinkPeer sinkPeer = new SinkPeer(serverNameA, mock(AdminService.BlockingInterface.class));
 
     sinkManager.reportBadSink(sinkPeer);
 
     // Just reporting a bad sink once shouldn't have an effect
-    assertEquals(1, sinkManager.getSinks().size());
+    assertEquals(1, sinkManager.getNumSinks());
 
   }
 
@@ -120,9 +120,9 @@ public class TestReplicationSinkManager {
 
     sinkManager.chooseSinks();
     // Sanity check
-    assertEquals(3, sinkManager.getSinks().size());
+    assertEquals(3, sinkManager.getNumSinks());
 
-    ServerName serverName = sinkManager.getSinks().get(0);
+    ServerName serverName = sinkManager.getSinksForTesting().get(0);
 
     SinkPeer sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
 
@@ -133,12 +133,12 @@ public class TestReplicationSinkManager {
 
     // Reporting a bad sink more than the threshold count should remove it
     // from the list of potential sinks
-    assertEquals(2, sinkManager.getSinks().size());
+    assertEquals(2, sinkManager.getNumSinks());
 
     //
     // now try a sink that has some successes
     //
-    serverName = sinkManager.getSinks().get(0);
+    serverName = sinkManager.getSinksForTesting().get(0);
 
     sinkPeer = new SinkPeer(serverName, mock(AdminService.BlockingInterface.class));
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD-1; i++) {
@@ -148,17 +148,17 @@ public class TestReplicationSinkManager {
     sinkManager.reportBadSink(sinkPeer);
 
     // did not remove the sink, since we had one successful try
-    assertEquals(2, sinkManager.getSinks().size());
+    assertEquals(2, sinkManager.getNumSinks());
 
     for (int i = 0; i <= ReplicationSinkManager.DEFAULT_BAD_SINK_THRESHOLD-2; i++) {
       sinkManager.reportBadSink(sinkPeer);
     }
     // still not remove, since the success reset the counter
-    assertEquals(2, sinkManager.getSinks().size());
+    assertEquals(2, sinkManager.getNumSinks());
 
     sinkManager.reportBadSink(sinkPeer);
     // but we exhausted the tries
-    assertEquals(1, sinkManager.getSinks().size());
+    assertEquals(1, sinkManager.getNumSinks());
   }
 
   @Test
@@ -174,7 +174,7 @@ public class TestReplicationSinkManager {
     sinkManager.chooseSinks();
     // Sanity check
 
-    List<ServerName> sinkList = sinkManager.getSinks();
+    List<ServerName> sinkList = sinkManager.getSinksForTesting();
     assertEquals(2, sinkList.size());
 
     ServerName serverNameA = sinkList.get(0);
@@ -190,7 +190,7 @@ public class TestReplicationSinkManager {
 
     // We've gone down to 0 good sinks, so the replication sinks
     // should have been refreshed now
-    assertEquals(2, sinkManager.getSinks().size());
+    assertEquals(2, sinkManager.getNumSinks());
   }
 
 }


[11/43] hbase git commit: HBASE-13907 Document how to deploy a coprocessor

Posted by sy...@apache.org.
HBASE-13907 Document how to deploy a coprocessor


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

Branch: refs/heads/hbase-12439
Commit: f8eab44dcd0d15ed5a4bf039c382f73468709a33
Parents: 7a4590d
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Tue Jun 16 14:13:00 2015 +1000
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Fri Dec 18 08:35:50 2015 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/cp.adoc | 707 +++++++++++++++----------------
 1 file changed, 338 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f8eab44d/src/main/asciidoc/_chapters/cp.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/cp.adoc b/src/main/asciidoc/_chapters/cp.adoc
index a4587ec..5f50b68 100644
--- a/src/main/asciidoc/_chapters/cp.adoc
+++ b/src/main/asciidoc/_chapters/cp.adoc
@@ -27,251 +27,209 @@
 :icons: font
 :experimental:
 
-HBase Coprocessors are modeled after the Coprocessors which are part of Google's BigTable
-(http://research.google.com/people/jeff/SOCC2010-keynote-slides.pdf pages 41-42.). +
-Coprocessor is a framework that provides an easy way to run your custom code directly on
-Region Server.
-The information in this chapter is primarily sourced and heavily reused from:
+HBase Coprocessors are modeled after Google BigTable's coprocessor implementation
+(http://research.google.com/people/jeff/SOCC2010-keynote-slides.pdf pages 41-42.).
+
+The coprocessor framework provides mechanisms for running your custom code directly on
+the RegionServers managing your data. Efforts are ongoing to bridge gaps between HBase's
+implementation and BigTable's architecture. For more information see
+link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047].
+
+The information in this chapter is primarily sourced and heavily reused from the following
+resources:
 
 . Mingjie Lai's blog post
 link:https://blogs.apache.org/hbase/entry/coprocessor_introduction[Coprocessor Introduction].
 . Gaurav Bhardwaj's blog post
 link:http://www.3pillarglobal.com/insights/hbase-coprocessors[The How To Of HBase Coprocessors].
 
+[WARNING]
+.Use Coprocessors At Your Own Risk
+====
+Coprocessors are an advanced feature of HBase and are intended to be used by system
+developers only. Because coprocessor code runs directly on the RegionServer and has
+direct access to your data, they introduce the risk of data corruption, man-in-the-middle
+attacks, or other malicious data access. Currently, there is no mechanism to prevent
+data corruption by coprocessors, though work is underway on
+link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047].
++
+In addition, there is no resource isolation, so a well-intentioned but misbehaving
+coprocessor can severely degrade cluster performance and stability.
+====
 
+== Coprocessor Overview
 
-== Coprocessor Framework
-
-When working with any data store (like RDBMS or HBase) you fetch the data (in case of RDBMS you
-might use SQL query and in case of HBase you use either Get or Scan). To fetch only relevant data
-you filter it (for RDBMS you put conditions in 'WHERE' predicate and in HBase you use
-link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html[Filter]).
-After fetching the desired data, you perform your business computation on the data.
-This scenario is close to ideal for "small data", where few thousand rows and a bunch of columns
-are returned from the data store. Now imagine a scenario where there are billions of rows and
-millions of columns and you want to perform some computation which requires all the data, like
-calculating average or sum. Even if you are interested in just few columns, you still have to
-fetch all the rows. There are a few drawbacks in this approach as described below:
-
-. In this approach the data transfer (from data store to client side) will become the bottleneck,
-and the time required to complete the operation is limited by the rate at which data transfer
-takes place.
-. It's not always possible to hold so much data in memory and perform computation.
-. Bandwidth is one of the most precious resources in any data center. Operations like this may
-saturate your data center’s bandwidth and will severely impact the performance of your cluster.
-. Your client code is becoming thick as you are maintaining the code for calculating average or
-summation on client side. Not a major drawback when talking of severe issues like
-performance/bandwidth but still worth giving consideration.
-
-In a scenario like this it's better to move the computation (i.e. user's custom code) to the data
-itself (Region Server). Coprocessor helps you achieve this but you can do more than that.
-There is another advantage that your code runs in parallel (i.e. on all Regions).
-To give an idea of Coprocessor's capabilities, different people give different analogies.
-The three most famous analogies for Coprocessor are:
-[[cp_analogies]]
-Triggers and Stored Procedure:: This is the most common analogy for Coprocessor. Observer
-Coprocessor is compared to triggers because like triggers they execute your custom code when
-certain event occurs (like Get or Put etc.). Similarly Endpoints Coprocessor is compared to the
-stored procedures and you can perform custom computation on data directly inside the region server.
+In HBase, you fetch data using a `Get` or `Scan`, whereas in an RDBMS you use a SQL
+query. In order to fetch only the relevant data, you filter it using a HBase
+link:http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/filter/Filter.html[Filter]
+, whereas in an RDBMS you use a `WHERE` predicate.
 
-MapReduce:: As in MapReduce you move the computation to the data in the same way. Coprocessor
-executes your custom computation directly on Region Servers, i.e. where data resides. That's why
-some people compare Coprocessor to a small MapReduce jobs.
+After fetching the data, you perform computations on it. This paradigm works well
+for "small data" with a few thousand rows and several columns. However, when you scale
+to billions of rows and millions of columns, moving large amounts of data across your
+network will create bottlenecks at the network layer, and the client needs to be powerful
+enough and have enough memory to handle the large amounts of data and the computations.
+In addition, the client code can grow large and complex.
 
-AOP:: Some people compare it to _Aspect Oriented Programming_ (AOP). As in AOP, you apply advice
-(on occurrence of specific event) by intercepting the request and then running some custom code
-(probably cross-cutting concerns) and then forwarding the request on its path as if nothing
-happened (or even return it back). Similarly in Coprocessor you have this facility of intercepting
-the request and running custom code and then forwarding it on its path (or returning it).
+In this scenario, coprocessors might make sense. You can put the business computation
+code into a coprocessor which runs on the RegionServer, in the same location as the
+data, and returns the result to the client.
 
+This is only one scenario where using coprocessors can provide benefit. Following
+are some analogies which may help to explain some of the benefits of coprocessors.
 
-Although Coprocessor derives its roots from Google's Bigtable but it deviates from it largely in
-its design. Currently there are efforts going on to bridge this gap. For more information see
-link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047].
+[[cp_analogies]]
+=== Coprocessor Analogies
 
-In HBase, to implement a Coprocessor certain steps must be followed as described below:
+Triggers and Stored Procedure::
+  An Observer coprocessor is similar to a trigger in a RDBMS in that it executes
+  your code either before or after a specific event (such as a `Get` or `Put`)
+  occurs. An endpoint coprocessor is similar to a stored procedure in a RDBMS
+  because it allows you to perform custom computations on the data on the
+  RegionServer itself, rather than on the client.
 
-. Either your class should extend one of the Coprocessor classes (like
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver]
-) or it should implement Coprocessor interfaces (like
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/Coprocessor.html[Coprocessor],
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/CoprocessorService.html[CoprocessorService]).
+MapReduce::
+  MapReduce operates on the principle of moving the computation to the location of
+  the data. Coprocessors operate on the same principal.
 
-. Load the Coprocessor: Currently there are two ways to load the Coprocessor. +
-Static:: Loading from configuration
-Dynamic:: Loading via 'hbase shell' or via Java code using HTableDescriptor class). +
-For more details see <<cp_loading,Loading Coprocessors>>.
+AOP::
+  If you are familiar with Aspect Oriented Programming (AOP), you can think of a coprocessor
+  as applying advice by intercepting a request and then running some custom code,
+  before passing the request on to its final destination (or even changing the destination).
 
-. Finally your client-side code to call the Coprocessor. This is the easiest step, as HBase
-handles the Coprocessor transparently and you don't have to do much to call the Coprocessor.
 
+=== Coprocessor Implementation Overview
 
-The framework API is provided in the
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor]
-package. +
-Coprocessors are not designed to be used by the end users but by developers. Coprocessors are
-executed directly on region server; therefore a faulty/malicious code can bring your region server
-down. Currently there is no mechanism to prevent this, but there are efforts going on for this.
-For more, see link:https://issues.apache.org/jira/browse/HBASE-4047[HBASE-4047]. +
-Two different types of Coprocessors are provided by the framework, based on their functionality.
+. Either your class should extend one of the Coprocessor classes, such as
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver],
+or it should implement the link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/Coprocessor.html[Coprocessor]
+or
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/CoprocessorService.html[CoprocessorService]
+interface.
 
+. Load the coprocessor, either statically (from the configuration) or dynamically,
+using HBase Shell. For more details see <<cp_loading,Loading Coprocessors>>.
 
+. Call the coprocessor from your client-side code. HBase handles the coprocessor
+trapsparently.
 
-== Types of Coprocessors
+The framework API is provided in the
+link:https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/coprocessor/package-summary.html[coprocessor]
+package.
 
-Coprocessor can be broadly divided into two categories: Observer and Endpoint.
-
-=== Observer
-Observer Coprocessor are easy to understand. People coming from RDBMS background can compare them
-to the triggers available in relational databases. Folks coming from programming background can
-visualize it like advice (before and after only) available in AOP (Aspect Oriented Programming).
-See <<cp_analogies, Coprocessor Analogy>> +
-Coprocessors allows you to hook your custom code in two places during the life cycle of an event. +
-First is just _before_ the occurrence of the event (just like 'before' advice in AOP or triggers
-like 'before update'). All methods providing this kind feature will start with the prefix `pre`. +
-For example if you want your custom code to get executed just before the `Put` operation, you can
-use the override the
-// Below URL is more than 100 characters long.
-link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#prePut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`prePut`]
-method of
-// Below URL is more than 100 characters long.
-link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionCoprocessor].
-This method has following signature:
-[source,java]
-----
-public void prePut (final ObserverContext e, final Put put, final WALEdit edit,final Durability
-durability) throws IOException;
-----
+== Types of Coprocessors
 
-Secondly, the Observer Coprocessor also provides hooks for your code to get executed just _after_
-the occurrence of the event (similar to after advice in AOP terminology or 'after update' triggers
-). The methods giving this functionality will start with the prefix `post`. For example, if you
-want your code to be executed after the 'Put' operation, you should consider overriding
-// Below URL is more than 100 characters long.
-link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#postPut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`postPut`]
-method of
-// Below URL is more than 100 characters long.
-link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionCoprocessor]:
-[source,java]
-----
-public void postPut(final ObserverContext e, final Put put, final WALEdit edit, final Durability
-durability) throws IOException;
-----
+=== Observer Coprocessors
 
-In short, the following conventions are generally followed: +
-Override _preXXX()_ method if you want your code to be executed just before the occurrence of the
-event. +
-Override _postXXX()_ method if you want your code to be executed just after the occurrence of the
-event. +
+Observer coprocessors are triggered either before or after a specific event occurs.
+Observers that happen before an event use methods that start with a `pre` prefix,
+such as link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#prePut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`prePut`]. Observers that happen just after an event override methods that start
+with a `post` prefix, such as link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html#postPut%28org.apache.hadoop.hbase.coprocessor.ObserverContext,%20org.apache.hadoop.hbase.client.Put,%20org.apache.hadoop.hbase.regionserver.wal.WALEdit,%20org.apache.hadoop.hbase.client.Durability%29[`postPut`].
 
-.Use Cases for Observer Coprocessors:
-Few use cases of the Observer Coprocessor are:
 
-. *Security*: Before performing any operation (like 'Get', 'Put') you can check for permission in
-the 'preXXX' methods.
+==== Use Cases for Observer Coprocessors
+Security::
+  Before performing a `Get` or `Put` operation, you can check for permission using
+  `preGet` or `prePut` methods.
 
-. *Referential Integrity*: Unlike traditional RDBMS, HBase doesn't have the concept of referential
-integrity (foreign key). Suppose for example you have a requirement that whenever you insert a
-record in 'users' table, a corresponding entry should also be created in 'user_daily_attendance'
-table. One way you could solve this is by using two 'Put' one for each table, this way you are
-throwing the responsibility (of the referential integrity) to the user. A better way is to use
-Coprocessor and overriding 'postPut' method in which you write the code to insert the record in
-'user_daily_attendance' table. This way client code is more lean and clean.
+Referential Integrity::
+  HBase does not directly support the RDBMS concept of refential integrity, also known
+  as foreign keys. You can use a coprocessor to enforce such integrity. For instance,
+  if you have a business rule that every insert to the `users` table must be followed
+  by a corresponding entry in the `user_daily_attendance` table, you could implement
+  a coprocessor to use the `prePut` method on `user` to insert a record into `user_daily_attendance`.
 
-. *Secondary Index*: Coprocessor can be used to maintain secondary indexes. For more information
-see link:http://wiki.apache.org/hadoop/Hbase/SecondaryIndexing[SecondaryIndexing].
+Secondary Indexes::
+  You can use a coprocessor to maintain secondary indexes. For more information, see
+  link:http://wiki.apache.org/hadoop/Hbase/SecondaryIndexing[SecondaryIndexing].
 
 
 ==== Types of Observer Coprocessor
 
-Observer Coprocessor comes in following flavors:
-
-. *RegionObserver*: This Coprocessor provides the facility to hook your code when the events on
-region are triggered. Most common example include 'preGet' and 'postGet' for 'Get' operation and
-'prePut' and 'postPut' for 'Put' operation. For exhaustive list of supported methods (events) see
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionObserver].
-
-. *Region Server Observer*: Provides hook for the events related to the RegionServer, such as
-stopping the RegionServer and performing operations before or after merges, commits, or rollbacks.
-For more details please refer
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.html[RegionServerObserver].
-
-. *Master Observer*: This observer provides hooks for DDL like operation, such as create, delete,
-modify table. For entire list of available methods see
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html[MasterObserver].
-
-. *WAL Observer*: Provides hooks for WAL (Write-Ahead-Log) related operation. It has only two
-method 'preWALWrite()' and 'postWALWrite()'. For more details see
-// Below URL is more than 100 characters long.
-link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/WALObserver.html[WALObserver].
-
-For example see <<cp_example,Examples>>
+RegionObserver::
+  A RegionObserver coprocessor allows you to observe events on a region, such as `Get`
+  and `Put` operations. See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionObserver.html[RegionObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver],
+  which implements the `RegionObserver` interface and will not break if new methods are added.
+
+RegionServerObserver::
+  A RegionServerObserver allows you to observe events related to the RegionServer's
+  operation, such as starting, stopping, or performing merges, commits, or rollbacks.
+  See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.html[RegionServerObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseMasterRegionServerObserver.html[BaseMasterRegionServerObserver]
+  which implements both `MasterObserver` and `RegionServerObserver` interfaces and
+  will not break if new methods are added.
+
+MasterOvserver::
+  A MasterObserver allows you to observe events related to the HBase Master, such
+  as table creation, deletion, or schema modification. See
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html[MasterObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseMasterRegionServerObserver.html[BaseMasterRegionServerObserver],
+  which implements both `MasterObserver` and `RegionServerObserver` interfaces and
+  will not break if new methods are added.
+
+WalObserver::
+  A WalObserver allows you to observe events related to writes to the Write-Ahead
+  Log (WAL). See
+  link:http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/WALObserver.html[WALObserver].
+  Consider overriding the convenience class
+  link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseWALObserver.html[BaseWALObserver],
+  which implements the `WalObserver` interface and will not break if new methods are added.
+
+<<cp_example,Examples>> provides working examples of observer coprocessors.
 
 
 === Endpoint Coprocessor
 
-Endpoint Coprocessor can be compared to stored procedure found in RDBMS.
-See <<cp_analogies, Coprocessor Analogy>>. They help in performing computation which is not
-possible either through Observer Coprocessor or otherwise. For example, calculating average or
-summation over the entire table that spans across multiple regions. They do so by providing a hook
-for your custom code and then running it across all regions. +
-With Endpoints Coprocessor you can create your own dynamic RPC protocol and thus can provide
-communication between client and region server, hence enabling you to run your custom code on
-region server (on each region of a table). +
-Unlike observer Coprocessor (where your custom code is
-executed transparently when events like 'Get' operation occurs), in Endpoint Coprocessor you have
-to explicitly invoke the Coprocessor by using the
-// Below URL is more than 100 characters long.
+Endpoint processors allow you to perform computation at the location of the data.
+See <<cp_analogies, Coprocessor Analogy>>. An example is the need to calculate a running
+average or summation for an entire table which spans hundreds of regions.
+
+In contract to observer coprocessors, where your code is run transparently, endpoint
+coprocessors must be explicitly invoked using the
 link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html#coprocessorService%28java.lang.Class,%20byte%5B%5D,%20byte%5B%5D,%20org.apache.hadoop.hbase.client.coprocessor.Batch.Call%29[CoprocessorService()]
 method available in
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html[Table]
-(or
-// Below URL is more than 100 characters long.
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTableInterface.html[HTableInterface]
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/Table.html[Table],
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTableInterface.html[HTableInterface],
 or
-link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable]).
+link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/HTable.html[HTable].
 
-From version 0.96, implementing Endpoint Coprocessor is not straight forward. Now it is done with
-the help of Google's Protocol Buffer. For more details on Protocol Buffer, please see
+Starting with HBase 0.96, endpoint coprocessors are implemented using Google Protocol
+Buffers (protobuf). For more details on protobuf, see Google's
 link:https://developers.google.com/protocol-buffers/docs/proto[Protocol Buffer Guide].
-Endpoints Coprocessor written in version 0.94 are not compatible with version 0.96 or later
-(for more details, see
-link:https://issues.apache.org/jira/browse/HBASE-5448[HBASE-5448]),
-so if you are upgrading your HBase cluster from version 0.94 (or before) to 0.96 (or later) you
-have to rewrite your Endpoint coprocessor.
-
-For example see <<cp_example,Examples>>
+Endpoints Coprocessor written in version 0.94 are not compatible with version 0.96 or later.
+See
+link:https://issues.apache.org/jira/browse/HBASE-5448[HBASE-5448]). To upgrade your
+HBase cluster from 0.94 or earlier to 0.96 or later, you need to reimplement your
+coprocessor.
 
+<<cp_example,Examples>> provides working examples of endpoint coprocessors.
 
 [[cp_loading]]
 == Loading Coprocessors
 
-_Loading  of Coprocessor refers to the process of making your custom Coprocessor implementation
-available to HBase, so that when a request comes in or an event takes place the desired
-functionality implemented in your custom code gets executed. +
-Coprocessor can be loaded broadly in two ways. One is static (loading through configuration files)
-and the other one is dynamic loading (using hbase shell or java code).
+To make your coprocessor available to HBase, it must be _loaded_, either statically
+(through the HBase configuration) or dynamically (using HBase Shell or the Java API).
 
 === Static Loading
-Static loading means that your Coprocessor will take effect only when you restart your HBase and
-there is a reason for it. In this you make changes 'hbase-site.xml' and therefore have to restart
-HBase for your changes to take place. +
-Following are the steps for loading Coprocessor statically.
 
-. Define the Coprocessor in hbase-site.xml: Define a <property> element which consist of two
-sub elements <name> and <value> respectively.
+Follow these steps to statically load your coprocessor. Keep in mind that you must
+restart HBase to unload a coprocessor that has been loaded statically.
+
+. Define the Coprocessor in _hbase-site.xml_, with a <property> element with a <name>
+and a <value> sub-element. The <name> should be one of the following:
 +
-.. <name> can have one of the following values:
+- `hbase.coprocessor.region.classes` for RegionObservers and Endpoints.
+- `hbase.coprocessor.wal.classes` for WALObservers.
+- `hbase.coprocessor.master.classes` for MasterObservers.
 +
-... 'hbase.coprocessor.region.classes' for RegionObservers and Endpoints.
-... 'hbase.coprocessor.wal.classes' for WALObservers.
-... 'hbase.coprocessor.master.classes' for MasterObservers.
-.. <value> must contain the fully qualified class name of your class implementing the Coprocessor.
+<value> must contain the fully-qualified class name of your coprocessor's implementation
+class.
 +
 For example to load a Coprocessor (implemented in class SumEndPoint.java) you have to create
 following entry in RegionServer's 'hbase-site.xml' file (generally located under 'conf' directory):
@@ -283,6 +241,7 @@ following entry in RegionServer's 'hbase-site.xml' file (generally located under
     <value>org.myname.hbase.coprocessor.endpoint.SumEndPoint</value>
 </property>
 ----
++
 If multiple classes are specified for loading, the class names must be comma-separated.
 The framework attempts to load all the configured classes using the default class loader.
 Therefore, the jar file must reside on the server-side HBase classpath.
@@ -297,34 +256,32 @@ When calling out to registered observers, the framework executes their callbacks
 sorted order of their priority. +
 Ties are broken arbitrarily.
 
-. Put your code on classpath of HBase: There are various ways to do so, like adding jars on
-classpath etc. One easy way to do this is to drop the jar (containing you code and all the
-dependencies) in 'lib' folder of the HBase installation.
-
-. Restart the HBase.
+. Put your code HBase's classpath. One easy way to do this is to drop the jar
+  (containing you code and all the dependencies) into the `lib/` directory in the
+  HBase installation.
 
+. Restart HBase.
 
-==== Unloading Static Coprocessor
-Unloading static Coprocessor is easy. Following are the steps:
 
-. Delete the Coprocessor's entry from the 'hbase-site.xml' i.e. remove the <property> tag.
+=== Static Unloading
 
-. Restart the Hbase.
+. Delete the coprocessor's <property> element, including sub-elements, from `hbase-site.xml`.
+. Restart HBase.
+. Optionally, remove the coprocessor's JAR file from the classpath or HBase's `lib/`
+  directory.
 
-. Optionally remove the Coprocessor jar file from the classpath (or from the lib directory if you
-copied it over there). Removing the coprocessor JARs from HBase’s classpath is a good practice.
 
 === Dynamic Loading
-Dynamic loading refers to the process of loading Coprocessor without restarting HBase. This may
-sound better than the static loading (and in some scenarios it may) but there is a caveat, dynamic
-loaded Coprocessor applies to the table only for which it was loaded while same is not true for
-static loading as it applies to all the tables. Due to this difference sometimes dynamically
-loaded Coprocessor are also called *Table Coprocessor* (as they applies only to a single table)
-while statically loaded Coprocessor are called *System Coprocessor* (as they applies to all the
-tables). +
-To dynamically load the Coprocessor you have to take the table offline hence during this time you
-won't be able to process any request involving this table. +
-There are three ways to dynamically load Coprocessor as shown below:
+
+You can also load a coprocessor dynamically, without restarting HBase. This may seem
+preferable to static loading, but dynamically loaded coprocessors are loaded on a
+per-table basis, and are only available to the table for which they were loaded. For
+this reason, dynamically loaded tables are sometimes called *Table Coprocessor*.
+
+In addition, dynamically loading a coprocessor acts as a schema change on the table,
+and the table must be taken offline to load the coprocessor.
+
+There are three ways to dynamically load Coprocessor.
 
 [NOTE]
 .Assumptions
@@ -332,26 +289,25 @@ There are three ways to dynamically load Coprocessor as shown below:
 The below mentioned instructions makes the following assumptions:
 
 * A JAR called `coprocessor.jar` contains the Coprocessor implementation along with all of its
-dependencies if any.
+dependencies.
 * The JAR is available in HDFS in some location like
 `hdfs://<namenode>:<port>/user/<hadoop-user>/coprocessor.jar`.
 ====
 
-. *Using Shell*: You can load the Coprocessor using the HBase shell as follows:
-.. Disable Table: Take table offline by disabling it. Suppose if the table name is 'users', then
-to disable it enter following command:
+==== Using HBase Shell
+
+. Disable the table using HBase Shell:
 +
 [source]
 ----
-hbase(main):001:0> disable 'users'
+hbase> disable 'users'
 ----
 
-.. Load the Coprocessor: The Coprocessor jar should be on HDFS and should be accessible to HBase,
-to load the Coprocessor use following command:
+. Load the Coprocessor, using a command like the following:
 +
 [source]
 ----
-hbase(main):002:0> alter 'users', METHOD => 'table_att', 'Coprocessor'=>'hdfs://<namenode>:<port>/
+hbase alter 'users', METHOD => 'table_att', 'Coprocessor'=>'hdfs://<namenode>:<port>/
 user/<hadoop-user>/coprocessor.jar| org.myname.hbase.Coprocessor.RegionObserverExample|1073741823|
 arg1=1,arg2=2'
 ----
@@ -370,30 +326,25 @@ observers registered at the same hook using priorities. This field can be left b
 case the framework will assign a default priority value.
 * Arguments (Optional): This field is passed to the Coprocessor implementation. This is optional.
 
-.. Enable the table: To enable table type following command:
+. Enable the table.
 +
 ----
 hbase(main):003:0> enable 'users'
 ----
-.. Verification: This is optional but generally good practice to see if your Coprocessor is
-loaded successfully. Enter following command:
+
+. Verify that the coprocessor loaded:
 +
 ----
 hbase(main):04:0> describe 'users'
 ----
 +
-You must see some output like this:
-+
-----
-DESCRIPTION ENABLED
-'users', {TABLE_ATTRIBUTES => {coprocessor$1 => true 'hdfs://<namenode>:<port>/user/<hadoop-user>/
-coprocessor.jar| org.myname.hbase.Coprocessor.RegionObserverExample|1073741823|'}, {NAME =>
-'personalDet'.....
-----
+The coprocessor should be listed in the `TABLE_ATTRIBUTES`.
 
+==== Using the Java API (all HBase versions)
+
+The following Java code shows how to use the `setValue()` method of `HTableDescriptor`
+to load a coprocessor on the `users` table.
 
-. *Using setValue()* method of HTableDescriptor: This is done entirely in Java as follows:
-+
 [source,java]
 ----
 TableName tableName = TableName.valueOf("users");
@@ -416,9 +367,11 @@ admin.modifyTable(tableName, hTableDescriptor);
 admin.enableTable(tableName);
 ----
 
-. *Using addCoprocessor()* method of HTableDescriptor: This method is available from 0.96 version
-onwards.
-+
+==== Using the Java API (HBase 0.96+ only)
+
+In HBase 0.96 and newer, the `addCoprocessor()` method of `HTableDescriptor` provides
+an easier way to load a coprocessor dynamically.
+
 [source,java]
 ----
 TableName tableName = TableName.valueOf("users");
@@ -439,26 +392,42 @@ admin.modifyTable(tableName, hTableDescriptor);
 admin.enableTable(tableName);
 ----
 
-====
 WARNING: There is no guarantee that the framework will load a given Coprocessor successfully.
 For example, the shell command neither guarantees a jar file exists at a particular location nor
 verifies whether the given class is actually contained in the jar file.
-====
 
 
-==== Unloading Dynamic Coprocessor
-. Using shell: Run following command from HBase shell to remove Coprocessor from a table.
+=== Dynamic Unloading
+
+==== Using HBase Shell
+
+. Disable the table.
++
+[source]
+----
+hbase> disable 'users'
+----
+
+. Alter the table to remove the coprocessor.
 +
 [source]
 ----
-hbase(main):003:0> alter 'users', METHOD => 'table_att_unset',
-hbase(main):004:0*   NAME => 'coprocessor$1'
+hbase> alter 'users', METHOD => 'table_att_unset', NAME => 'coprocessor$1'
 ----
 
-. Using HTableDescriptor: Simply reload the table definition _without_ setting the value of
-Coprocessor either in setValue() or addCoprocessor() methods. This will remove the Coprocessor
-attached to this table, if any. For example:
+. Enable the table.
 +
+[source]
+----
+hbase> enable 'users'
+----
+
+==== Using the Java API
+
+Reload the table definition without setting the value of the coprocessor either by
+using `setValue()` or `addCoprocessor()` methods. This will remove any coprocessor
+attached to the table.
+
 [source,java]
 ----
 TableName tableName = TableName.valueOf("users");
@@ -477,26 +446,23 @@ hTableDescriptor.addFamily(columnFamily2);
 admin.modifyTable(tableName, hTableDescriptor);
 admin.enableTable(tableName);
 ----
-+
-Optionally you can also use removeCoprocessor() method of HTableDescriptor class.
 
+In HBase 0.96 and newer, you can instead use the `removeCoprocessor()` method of the
+`HTableDescriptor` class.
 
 
 [[cp_example]]
 == Examples
-HBase ships Coprocessor examples for Observer Coprocessor see
-// Below URL is more than 100 characters long.
+HBase ships examples for Observer Coprocessor in
 link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.html[ZooKeeperScanPolicyObserver]
-and for Endpoint Coprocessor see
-// Below URL is more than 100 characters long.
+and for Endpoint Coprocessor in
 link:http://hbase.apache.org/xref/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.html[RowCountEndpoint]
 
 A more detailed example is given below.
 
-For the sake of example let's take an hypothetical case. Suppose there is a HBase table called
-'users'. The table has two column families 'personalDet' and 'salaryDet' containing personal
-details and salary details respectively. Below is the graphical representation of the 'users'
-table.
+These examples assume a table called `users`, which has two column families `personalDet`
+and `salaryDet`, containing personal and salary details. Below is the graphical representation
+of the `users` table.
 
 .Users Table
 [width="100%",cols="7",options="header,footer"]
@@ -509,26 +475,22 @@ table.
 |====================
 
 
-
 === Observer Example
-For the purpose of demonstration of Coprocessor we are assuming that 'admin' is a special person
-and his details shouldn't be visible or returned to any client querying the 'users' table. +
-To implement this functionality we will take the help of Observer Coprocessor.
-Following are the implementation steps:
+
+The following Observer coprocessor prevents the details of the user `admin` from being
+returned in a `Get` or `Scan` of the `users` table.
 
 . Write a class that extends the
 link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.html[BaseRegionObserver]
 class.
 
-. Override the 'preGetOp()' method (Note that 'preGet()' method is now deprecated). The reason for
-overriding this method is to check if the client has queried for the rowkey with value 'admin' or
-not. If the client has queried rowkey with 'admin' value then return the call without allowing the
-system to perform the get operation thus saving on performance, otherwise process the request as
-normal.
+. Override the `preGetOp()` method (the `preGet()` method is deprecated) to check
+whether the client has queried for the rowkey with value `admin`. If so, return an
+empty result. Otherwise, process the request as normal.
 
-. Put your code and dependencies in the jar file.
+. Put your code and dependencies in a JAR file.
 
-. Place the jar in HDFS where HBase can locate it.
+. Place the JAR in HDFS where HBase can locate it.
 
 . Load the Coprocessor.
 
@@ -536,8 +498,7 @@ normal.
 
 Following are the implementation of the above steps:
 
-. For Step 1 and Step 2, below is the code.
-+
+
 [source,java]
 ----
 public class RegionObserverExample extends BaseRegionObserver {
@@ -568,10 +529,10 @@ public class RegionObserverExample extends BaseRegionObserver {
     }
 }
 ----
-Overriding the 'preGetOp()' will only work for 'Get' operation. For 'Scan' operation it won't help
-you. To deal with it you have to override another method called 'preScannerOpen()' method, and
-add a Filter explicitly for admin as shown below:
-+
+
+Overriding the `preGetOp()` will only work for `Get` operations. You also need to override
+the `preScannerOpen()` method to filter the `admin` row from scan results.
+
 [source,java]
 ----
 @Override
@@ -583,12 +544,11 @@ final RegionScanner s) throws IOException {
     return s;
 }
 ----
-+
-This method works but there is a _side effect_. If the client has used any Filter in his scan,
-then that Filter won't have any effect because our filter has replaced it. +
-Another option you can try is to deliberately remove the admin from result. This approach is
-shown below:
-+
+
+This method works but there is a _side effect_. If the client has used a filter in
+its scan, that filter will be replaced by this filter. Instead, you can explicitly
+remove any `admin` results from the scan:
+
 [source,java]
 ----
 @Override
@@ -597,9 +557,9 @@ final List results, final int limit, final boolean hasMore) throws IOException {
 	Result result = null;
     Iterator iterator = results.iterator();
     while (iterator.hasNext()) {
-		result = iterator.next();
+    result = iterator.next();
         if (Bytes.equals(result.getRow(), ROWKEY)) {
-			iterator.remove();
+            iterator.remove();
             break;
         }
     }
@@ -607,76 +567,12 @@ final List results, final int limit, final boolean hasMore) throws IOException {
 }
 ----
 
-. Step 3: It's pretty convenient to export the above program in a jar file. Let's assume that was
-exported in a file called 'coprocessor.jar'.
-
-. Step 4: Copy the jar to HDFS. You may use command like this:
-+
-[source]
-----
-hadoop fs -copyFromLocal coprocessor.jar coprocessor.jar
-----
-
-. Step 5: Load the Coprocessor, see <<cp_loading,Loading of Coprocessor>>.
-
-. Step 6: Run the following program to test. The first part is testing 'Get' and second 'Scan'.
-+
-[source,java]
-----
-Configuration conf = HBaseConfiguration.create();
-// Use below code for HBase version 1.x.x or above.
-Connection connection = ConnectionFactory.createConnection(conf);
-TableName tableName = TableName.valueOf("users");
-Table table = connection.getTable(tableName);
-
-//Use below code HBase version 0.98.xx or below.
-//HConnection connection = HConnectionManager.createConnection(conf);
-//HTableInterface table = connection.getTable("users");
-
-Get get = new Get(Bytes.toBytes("admin"));
-Result result = table.get(get);
-for (Cell c : result.rawCells()) {
-    System.out.println(Bytes.toString(CellUtil.cloneRow(c))
-        + "==> " + Bytes.toString(CellUtil.cloneFamily(c))
-        + "{" + Bytes.toString(CellUtil.cloneQualifier(c))
-        + ":" + Bytes.toLong(CellUtil.cloneValue(c)) + "}");
-}
-Scan scan = new Scan();
-ResultScanner scanner = table.getScanner(scan);
-for (Result res : scanner) {
-    for (Cell c : res.rawCells()) {
-        System.out.println(Bytes.toString(CellUtil.cloneRow(c))
-        + " ==> " + Bytes.toString(CellUtil.cloneFamily(c))
-        + " {" + Bytes.toString(CellUtil.cloneQualifier(c))
-        + ":" + Bytes.toLong(CellUtil.cloneValue(c))
-        + "}");
-    }
-}
-----
-
 === Endpoint Example
 
-In our hypothetical example (See Users Table), to demonstrate the Endpoint Coprocessor we see a
-trivial use case in which we will try to calculate the total (Sum) of gross salary of all
-employees. One way of implementing Endpoint Coprocessor (for version 0.96 and above) is as follows:
+Still using the `users` table, this example implements a coprocessor to calculate
+the sum of all employee salaries, using an endpoint coprocessor.
 
 . Create a '.proto' file defining your service.
-
-. Execute the 'protoc' command to generate the Java code from the above '.proto' file.
-
-. Write a class that should:
-.. Extend the above generated service class.
-.. It should also implement two interfaces Coprocessor and CoprocessorService.
-.. Override the service method.
-
-. Load the Coprocessor.
-
-. Write a client code to call Coprocessor.
-
-Implementation detail of the above steps is as follows:
-
-. Step 1: Create a 'proto' file to define your service, request and response. Let's call this file
-"sum.proto". Below is the content of the 'sum.proto' file.
 +
 [source]
 ----
@@ -700,26 +596,25 @@ service SumService {
 }
 ----
 
-. Step 2: Compile the proto file using proto compiler (for detailed instructions see the
-link:https://developers.google.com/protocol-buffers/docs/overview[official documentation]).
+. Execute the `protoc` command to generate the Java code from the above .proto' file.
 +
 [source]
 ----
+$ mkdir src
 $ protoc --java_out=src ./sum.proto
 ----
 +
-[note]
-----
-(Note: It is necessary for you to create the src folder).
-This will generate a class call "Sum.java".
-----
+This will generate a class call `Sum.java`.
 
-. Step 3: Write your Endpoint Coprocessor: Firstly your class should extend the service just
-defined above (i.e. Sum.SumService). Second it should implement Coprocessor and CoprocessorService
-interfaces. Third, override the 'getService()', 'start()', 'stop()' and 'getSum()' methods.
-Below is the full code:
+. Write a class that extends the generated service class, implement the `Coprocessor`
+and `CoprocessorService` classes, and override the service method.
 +
-[source,java]
+WARNING: If you load a coprocessor from `hbase-site.xml` and then load the same coprocessor
+again using HBase Shell, it will be loaded a second time. The same class will
+exist twice, and the second instance will have a higher ID (and thus a lower priority).
+The effect is that the duplicate coprocessor is effectively ignored.
++
+[source, java]
 ----
 public class SumEndPoint extends SumService implements Coprocessor, CoprocessorService {
 
@@ -779,15 +674,9 @@ public class SumEndPoint extends SumService implements Coprocessor, CoprocessorS
     }
 }
 ----
-
-. Step 4: Load the Coprocessor. See <<cp_loading,loading of Coprocessor>>.
-
-. Step 5: Now we have to write the client code to test it. To do so in your main method, write the
-following code as shown below:
 +
-[source,java]
+[source, java]
 ----
-
 Configuration conf = HBaseConfiguration.create();
 // Use below code for HBase version 1.x.x or above.
 Connection connection = ConnectionFactory.createConnection(conf);
@@ -821,6 +710,86 @@ e.printStackTrace();
 }
 ----
 
+. Load the Coprocessor.
+
+. Write a client code to call the Coprocessor.
+
+
+== Guidelines For Deploying A Coprocessor
+
+Bundling Coprocessors::
+  You can bundle all classes for a coprocessor into a
+  single JAR on the RegionServer's classpath, for easy deployment. Otherwise,
+  place all dependencies  on the RegionServer's classpath so that they can be
+  loaded during RegionServer start-up.  The classpath for a RegionServer is set
+  in the RegionServer's `hbase-env.sh` file.
+Automating Deployment::
+  You can use a tool such as Puppet, Chef, or
+  Ansible to ship the JAR for the coprocessor  to the required location on your
+  RegionServers' filesystems and restart each RegionServer,  to automate
+  coprocessor deployment. Details for such set-ups are out of scope of  this
+  document.
+Updating a Coprocessor::
+  Deploying a new version of a given coprocessor is not as simple as disabling it,
+  replacing the JAR, and re-enabling the coprocessor. This is because you cannot
+  reload a class in a JVM unless you delete all the current references to it.
+  Since the current JVM has reference to the existing coprocessor, you must restart
+  the JVM, by restarting the RegionServer, in order to replace it. This behavior
+  is not expected to change.
+Coprocessor Logging::
+  The Coprocessor framework does not provide an API for logging beyond standard Java
+  logging.
+Coprocessor Configuration::
+  If you do not want to load coprocessors from the HBase Shell, you can add their configuration
+  properties to `hbase-site.xml`. In <<load_coprocessor_in_shell>>, two arguments are
+  set: `arg1=1,arg2=2`. These could have been added to `hbase-site.xml` as follows:
+[source,xml]
+----
+<property>
+  <name>arg1</name>
+  <value>1</value>
+</property>
+<property>
+  <name>arg2</name>
+  <value>2</value>
+</property>
+----
+Then you can read the configuration using code like the following:
+[source,java]
+----
+Configuration conf = HBaseConfiguration.create();
+// Use below code for HBase version 1.x.x or above.
+Connection connection = ConnectionFactory.createConnection(conf);
+TableName tableName = TableName.valueOf("users");
+Table table = connection.getTable(tableName);
+
+//Use below code HBase version 0.98.xx or below.
+//HConnection connection = HConnectionManager.createConnection(conf);
+//HTableInterface table = connection.getTable("users");
+
+Get get = new Get(Bytes.toBytes("admin"));
+Result result = table.get(get);
+for (Cell c : result.rawCells()) {
+    System.out.println(Bytes.toString(CellUtil.cloneRow(c))
+        + "==> " + Bytes.toString(CellUtil.cloneFamily(c))
+        + "{" + Bytes.toString(CellUtil.cloneQualifier(c))
+        + ":" + Bytes.toLong(CellUtil.cloneValue(c)) + "}");
+}
+Scan scan = new Scan();
+ResultScanner scanner = table.getScanner(scan);
+for (Result res : scanner) {
+    for (Cell c : res.rawCells()) {
+        System.out.println(Bytes.toString(CellUtil.cloneRow(c))
+        + " ==> " + Bytes.toString(CellUtil.cloneFamily(c))
+        + " {" + Bytes.toString(CellUtil.cloneQualifier(c))
+        + ":" + Bytes.toLong(CellUtil.cloneValue(c))
+        + "}");
+    }
+}
+----
+
+
+
 
 == Monitor Time Spent in Coprocessors
 


[25/43] hbase git commit: HBASE-15021 hadoopqa doing false positives

Posted by sy...@apache.org.
HBASE-15021 hadoopqa doing false positives


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

Branch: refs/heads/hbase-12439
Commit: 53e5d27a80081c23919392a106ddae94be1ac16c
Parents: f018c37
Author: stack <st...@apache.org>
Authored: Mon Dec 21 22:16:06 2015 -0800
Committer: stack <st...@apache.org>
Committed: Mon Dec 21 22:16:38 2015 -0800

----------------------------------------------------------------------
 dev-support/test-patch.sh | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/53e5d27a/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index 9366319..509edeb 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -841,14 +841,13 @@ runTests () {
   echo "======================================================================"
   echo ""
   echo ""
-
   failed_tests=""
   echo "$MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess"
   export MAVEN_OPTS="${MAVEN_OPTS}"
   ulimit -a
-  $MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess
   # Need to export this so the zombie subshell picks up current content
   export JIRA_COMMENT
+  $MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess
   if [[ $? != 0 ]] ; then
      ### Find and format names of failed tests
      failed_tests=`find . -name 'TEST*.xml' | xargs $GREP  -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-|                  |g" | sed -e "s|\.xml||g"`


[43/43] hbase git commit: HBASE-14800 TCompareOp which was missing from initial checkin

Posted by sy...@apache.org.
HBASE-14800 TCompareOp which was missing from initial checkin


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

Branch: refs/heads/hbase-12439
Commit: 5eefe1317336714b409500daaebe003f65fd87b6
Parents: b8e2e4a
Author: tedyu <yu...@gmail.com>
Authored: Fri Dec 25 18:13:35 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Dec 25 18:13:35 2015 -0800

----------------------------------------------------------------------
 .../hbase/thrift2/generated/TCompareOp.java     | 64 ++++++++++++++++++++
 1 file changed, 64 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5eefe131/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
new file mode 100644
index 0000000..efbba09
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCompareOp.java
@@ -0,0 +1,64 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.filter.CompareFilter$CompareOp.
+ */
+public enum TCompareOp implements org.apache.thrift.TEnum {
+  LESS(0),
+  LESS_OR_EQUAL(1),
+  EQUAL(2),
+  NOT_EQUAL(3),
+  GREATER_OR_EQUAL(4),
+  GREATER(5),
+  NO_OP(6);
+
+  private final int value;
+
+  private TCompareOp(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static TCompareOp findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return LESS;
+      case 1:
+        return LESS_OR_EQUAL;
+      case 2:
+        return EQUAL;
+      case 3:
+        return NOT_EQUAL;
+      case 4:
+        return GREATER_OR_EQUAL;
+      case 5:
+        return GREATER;
+      case 6:
+        return NO_OP;
+      default:
+        return null;
+    }
+  }
+}


[34/43] hbase git commit: HBASE-15034 IntegrationTestDDLMasterFailover does not clean created namespaces.

Posted by sy...@apache.org.
HBASE-15034 IntegrationTestDDLMasterFailover does not clean created namespaces.

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/hbase-12439
Commit: 0bdd6e487fa89ff614593cd3a32e690c1d380e22
Parents: 6fc2596
Author: Samir Ahmic <sa...@personal.com>
Authored: Wed Dec 23 23:30:26 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Dec 24 04:53:20 2015 -0800

----------------------------------------------------------------------
 .../hbase/IntegrationTestDDLMasterFailover.java   | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0bdd6e48/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
index d9c7caa..4b75ce2 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestDDLMasterFailover.java
@@ -103,7 +103,7 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
 
   protected static final int DEFAULT_NUM_REGIONS = 50; // number of regions in pre-split tables
 
-  private boolean keepTableAtTheEnd = false;
+  private boolean keepObjectsAtTheEnd = false;
   protected HBaseCluster cluster;
 
   protected Connection connection;
@@ -144,11 +144,19 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
 
   @Override
   public void cleanUpCluster() throws Exception {
-    if (!keepTableAtTheEnd) {
-      Admin admin = util.getHBaseAdmin();
+    if (!keepObjectsAtTheEnd) {
+      Admin admin = util.getAdmin();
       admin.disableTables("ittable-\\d+");
       admin.deleteTables("ittable-\\d+");
+      NamespaceDescriptor [] nsds = admin.listNamespaceDescriptors();
+      for(NamespaceDescriptor nsd:nsds ) {
+        if(nsd.getName().matches("itnamespace\\d+")) {
+          LOG.info("Removing namespace="+nsd.getName());
+          admin.deleteNamespace(nsd.getName());
+        }
+      }
     }
+
     enabledTables.clear();
     disabledTables.clear();
     deletedTables.clear();
@@ -938,9 +946,9 @@ public class IntegrationTestDDLMasterFailover extends IntegrationTestBase {
       LOG.info("Running hbck");
       hbck = HbckTestingUtil.doFsck(util.getConfiguration(), false);
       if (HbckTestingUtil.inconsistencyFound(hbck)) {
-        // Find the inconsistency during HBCK. Leave table undropped so that
+        // Find the inconsistency during HBCK. Leave table and namespace undropped so that
         // we can check outside the test.
-        keepTableAtTheEnd = true;
+        keepObjectsAtTheEnd = true;
       }
       HbckTestingUtil.assertNoErrors(hbck);
       LOG.info("Finished hbck");


[27/43] hbase git commit: HBASE-14977 ChoreService.shutdown may result in ConcurrentModificationException - ADDENDUM to fix extra concurrency issues

Posted by sy...@apache.org.
HBASE-14977 ChoreService.shutdown may result in ConcurrentModificationException - ADDENDUM to fix extra concurrency issues


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

Branch: refs/heads/hbase-12439
Commit: 95a13b51ee052eb73882682e8f009bfa1e914866
Parents: 6e2c5d2
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Dec 22 02:04:09 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Dec 22 02:04:09 2015 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ScheduledChore.java    | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/95a13b51/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
index 538b390..dd98d26 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
@@ -228,7 +228,7 @@ public abstract class ScheduledChore implements Runnable {
         && getTimeBetweenRuns() > getMaximumAllowedTimeBetweenRuns();
   }
 
-  private synchronized double getMaximumAllowedTimeBetweenRuns() {
+  private double getMaximumAllowedTimeBetweenRuns() {
     // Threshold used to determine if the Chore's current run started too late
     return 1.5 * period;
   }
@@ -268,23 +268,23 @@ public abstract class ScheduledChore implements Runnable {
     choreServicer = null;
   }
 
-  public synchronized String getName() {
+  public String getName() {
     return name;
   }
 
-  public synchronized Stoppable getStopper() {
+  public Stoppable getStopper() {
     return stopper;
   }
 
-  public synchronized int getPeriod() {
+  public int getPeriod() {
     return period;
   }
 
-  public synchronized long getInitialDelay() {
+  public long getInitialDelay() {
     return initialDelay;
   }
 
-  public final synchronized TimeUnit getTimeUnit() {
+  public TimeUnit getTimeUnit() {
     return timeUnit;
   }
 


[09/43] hbase git commit: HBASE-15005 Use value array in computing block length for 1.2 and 1.3

Posted by sy...@apache.org.
HBASE-15005 Use value array in computing block length for 1.2 and 1.3


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

Branch: refs/heads/hbase-12439
Commit: 4bfeccb87a94cfe232ea8fc9a6f40ff5b8d3b1c5
Parents: 408666a
Author: Elliott Clark <ec...@apache.org>
Authored: Thu Dec 17 23:36:43 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Fri Dec 18 00:23:02 2015 -0800

----------------------------------------------------------------------
 .../hbase/client/TestMultiRespectsLimits.java   | 20 +++++++++++++++-----
 1 file changed, 15 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4bfeccb8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
index 28e1855..04c592e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
@@ -20,9 +20,12 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.metrics.BaseSource;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -110,7 +113,12 @@ public class TestMultiRespectsLimits {
   @Test
   public void testBlockMultiLimits() throws Exception {
     final TableName name = TableName.valueOf("testBlockMultiLimits");
-    Table t = TEST_UTIL.createTable(name, FAMILY);
+    HTableDescriptor desc = new HTableDescriptor(name);
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+    hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
+    desc.addFamily(hcd);
+    TEST_UTIL.getHBaseAdmin().createTable(desc);
+    Table t = TEST_UTIL.getConnection().getTable(name);
 
     final HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0);
     RpcServerInterface rpcServer = regionServer.getRpcServer();
@@ -122,14 +130,16 @@ public class TestMultiRespectsLimits {
     byte[][] cols = new byte[][]{
         Bytes.toBytes("0"), // Get this
         Bytes.toBytes("1"), // Buffer
-        Bytes.toBytes("2"), // Get This
-        Bytes.toBytes("3"), // Buffer
+        Bytes.toBytes("2"), // Buffer
+        Bytes.toBytes("3"), // Get This
+        Bytes.toBytes("4"), // Buffer
+        Bytes.toBytes("5"), // Buffer
     };
 
     // Set the value size so that one result will be less than the MAX_SIE
     // however the block being reference will be larger than MAX_SIZE.
     // This should cause the regionserver to try and send a result immediately.
-    byte[] value = new byte[MAX_SIZE - 200];
+    byte[] value = new byte[MAX_SIZE - 100];
     ThreadLocalRandom.current().nextBytes(value);
 
     for (byte[] col:cols) {
@@ -155,7 +165,7 @@ public class TestMultiRespectsLimits {
     gets.add(g0);
 
     Get g2 = new Get(row);
-    g2.addColumn(FAMILY, cols[2]);
+    g2.addColumn(FAMILY, cols[3]);
     gets.add(g2);
 
     Result[] results = t.get(gets);


[36/43] hbase git commit: HBASE-14717 enable_table_replication command should only create specified table for a peer cluster (Ashish)

Posted by sy...@apache.org.
HBASE-14717 enable_table_replication command should only create specified table for a peer cluster (Ashish)


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

Branch: refs/heads/hbase-12439
Commit: a1a19d94059dc3750b477ca03f89a77d53224655
Parents: e15c48e
Author: tedyu <yu...@gmail.com>
Authored: Thu Dec 24 11:07:49 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Dec 24 11:07:49 2015 -0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    | 13 ++++-
 .../replication/ReplicationPeerZKImpl.java      | 20 +++++++-
 .../TestReplicationAdminWithClusters.java       | 51 +++++++++++++++++---
 3 files changed, 73 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a1a19d94/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index a0bea8b..c2e7489 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -599,7 +599,17 @@ public class ReplicationAdmin implements Closeable {
     if (repPeers == null || repPeers.size() <= 0) {
       throw new IllegalArgumentException("Found no peer cluster for replication.");
     }
+    
+    final TableName onlyTableNameQualifier = TableName.valueOf(tableName.getQualifierAsString());
+    
     for (ReplicationPeer repPeer : repPeers) {
+      Map<TableName, List<String>> tableCFMap = repPeer.getTableCFs();
+      // TODO Currently peer TableCFs will not include namespace so we need to check only for table
+      // name without namespace in it. Need to correct this logic once we fix HBASE-11386.
+      if (tableCFMap != null && !tableCFMap.containsKey(onlyTableNameQualifier)) {
+        continue;
+      }
+
       Configuration peerConf = repPeer.getConfiguration();
       HTableDescriptor htd = null;
       try (Connection conn = ConnectionFactory.createConnection(peerConf);
@@ -638,7 +648,8 @@ public class ReplicationAdmin implements Closeable {
       try {
         Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
         Configuration peerConf = pair.getSecond();
-        ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
+        ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst(),
+            parseTableCFsFromConfig(this.getPeerTableCFs(peerId)));
         s =
             zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
               null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1a19d94/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 3ac8007..39f6ebc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -55,8 +55,8 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
   private TableCFsTracker tableCFsTracker;
 
   /**
-   * Constructor that takes all the objects required to communicate with the
-   * specified peer, except for the region server addresses.
+   * Constructor that takes all the objects required to communicate with the specified peer, except
+   * for the region server addresses.
    * @param conf configuration object to this peer
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
@@ -67,6 +67,22 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
     this.peerConfig = peerConfig;
     this.id = id;
   }
+  
+  /**
+   * Constructor that takes all the objects required to communicate with the specified peer, except
+   * for the region server addresses.
+   * @param conf configuration object to this peer
+   * @param id string representation of this peer's identifier
+   * @param peerConfig configuration for the replication peer
+   * @param tableCFs table-cf configuration for this peer
+   */
+  public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
+      Map<TableName, List<String>> tableCFs) throws ReplicationException {
+    this.conf = conf;
+    this.peerConfig = peerConfig;
+    this.id = id;
+    this.tableCFs = tableCFs;
+  }
 
   /**
    * start a state tracker to check whether this peer is enabled or not

http://git-wip-us.apache.org/repos/asf/hbase/blob/a1a19d94/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index b5899b8..e7bd72c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -15,6 +15,10 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -41,6 +45,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
   static Connection connection2;
   static Admin admin1;
   static Admin admin2;
+  static ReplicationAdmin adminExt;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -49,12 +54,14 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     connection2 = ConnectionFactory.createConnection(conf2);
     admin1 = connection1.getAdmin();
     admin2 = connection2.getAdmin();
+    adminExt = new ReplicationAdmin(conf1);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     admin1.close();
     admin2.close();
+    adminExt.close();
     connection1.close();
     connection2.close();
     TestReplicationBase.tearDownAfterClass();
@@ -65,7 +72,6 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     admin2.disableTable(tableName);
     admin2.deleteTable(tableName);
     assertFalse(admin2.tableExists(tableName));
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.enableTableRep(tableName);
     assertTrue(admin2.tableExists(tableName));
   }
@@ -84,7 +90,6 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     admin2.modifyTable(tableName, table);
     admin2.enableTable(tableName);
 
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.enableTableRep(tableName);
     table = admin1.getTableDescriptor(tableName);
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
@@ -101,7 +106,6 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
     admin2.modifyTable(tableName, table);
     admin2.enableTable(tableName);
 
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     try {
       adminExt.enableTableRep(tableName);
       fail("Exception should be thrown if table descriptors in the clusters are not same.");
@@ -120,7 +124,6 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000)
   public void testDisableAndEnableReplication() throws Exception {
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.disableTableRep(tableName);
     HTableDescriptor table = admin1.getTableDescriptor(tableName);
     for (HColumnDescriptor fam : table.getColumnFamilies()) {
@@ -139,25 +142,57 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000, expected = TableNotFoundException.class)
   public void testDisableReplicationForNonExistingTable() throws Exception {
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.disableTableRep(TableName.valueOf("nonExistingTable"));
   }
 
   @Test(timeout = 300000, expected = TableNotFoundException.class)
   public void testEnableReplicationForNonExistingTable() throws Exception {
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.enableTableRep(TableName.valueOf("nonExistingTable"));
   }
 
   @Test(timeout = 300000, expected = IllegalArgumentException.class)
   public void testDisableReplicationWhenTableNameAsNull() throws Exception {
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.disableTableRep(null);
   }
 
   @Test(timeout = 300000, expected = IllegalArgumentException.class)
   public void testEnableReplicationWhenTableNameAsNull() throws Exception {
-    ReplicationAdmin adminExt = new ReplicationAdmin(conf1);
     adminExt.enableTableRep(null);
   }
+  
+  /*
+   * Test enable table replication should create table only in user explicit specified table-cfs.
+   * HBASE-14717
+   */
+  @Test(timeout = 300000)
+  public void testEnableReplicationForExplicitSetTableCfs() throws Exception {
+    TableName tn = TableName.valueOf("testEnableReplicationForSetTableCfs");
+    String peerId = "2";
+    if (admin2.isTableAvailable(tableName)) {
+      admin2.disableTable(tableName);
+      admin2.deleteTable(tableName);
+    }
+    assertFalse("Table should not exists in the peer cluster", admin2.isTableAvailable(tableName));
+
+    Map<TableName, ? extends Collection<String>> tableCfs =
+        new HashMap<TableName, Collection<String>>();
+    tableCfs.put(tn, null);
+    try {
+      adminExt.setPeerTableCFs(peerId, tableCfs);
+      adminExt.enableTableRep(tableName);
+      assertFalse("Table should not be created if user has set table cfs explicitly for the "
+          + "peer and this is not part of that collection",
+        admin2.isTableAvailable(tableName));
+
+      tableCfs.put(tableName, null);
+      adminExt.setPeerTableCFs(peerId, tableCfs);
+      adminExt.enableTableRep(tableName);
+      assertTrue(
+        "Table should be created if user has explicitly added table into table cfs collection",
+        admin2.isTableAvailable(tableName));
+    } finally {
+      adminExt.removePeerTableCFs(peerId, adminExt.getPeerTableCFs(peerId));
+      adminExt.disableTableRep(tableName);
+    }
+  }
 }


[29/43] hbase git commit: HBASE-14980 Project Astro; addendum

Posted by sy...@apache.org.
HBASE-14980 Project Astro; addendum


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

Branch: refs/heads/hbase-12439
Commit: 1af98f255132ef6716a1f6ba1d8d71a36ea38840
Parents: 2439f7a
Author: stack <st...@apache.org>
Authored: Tue Dec 22 08:11:33 2015 -0800
Committer: stack <st...@apache.org>
Committed: Tue Dec 22 08:11:50 2015 -0800

----------------------------------------------------------------------
 src/main/site/xdoc/poweredbyhbase.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1af98f25/src/main/site/xdoc/poweredbyhbase.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/poweredbyhbase.xml b/src/main/site/xdoc/poweredbyhbase.xml
index c940109..ff1ba59 100644
--- a/src/main/site/xdoc/poweredbyhbase.xml
+++ b/src/main/site/xdoc/poweredbyhbase.xml
@@ -42,7 +42,7 @@ under the License.
     write data to HBase and run mapreduce jobs to process then store it back to
     HBase or external systems. Our production cluster has been running since Oct 2008.</dd>
 
-  <dt><a href="http://huaweibigdata.github.io/astro/">Project Astro</a></dt>
+  <dt><a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Project Astro</a></dt>
   <dd>
     Astro provides fast Spark SQL/DataFrame capabilities to HBase data,
     featuring super-efficient access to multi-dimensional HBase rows through


[42/43] hbase git commit: HBASE-14800 Expose checkAndMutate via Thrift2 (Josh Elser)

Posted by sy...@apache.org.
HBASE-14800 Expose checkAndMutate via Thrift2 (Josh Elser)


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

Branch: refs/heads/hbase-12439
Commit: b8e2e4a1e7c275752177baf64d6bf01e0da8228c
Parents: f53542c
Author: tedyu <yu...@gmail.com>
Authored: Fri Dec 25 17:41:37 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Dec 25 17:41:37 2015 -0800

----------------------------------------------------------------------
 .../hbase/thrift/generated/AlreadyExists.java   |    2 +-
 .../hbase/thrift/generated/BatchMutation.java   |    2 +-
 .../thrift/generated/ColumnDescriptor.java      |    2 +-
 .../hadoop/hbase/thrift/generated/Hbase.java    |    2 +-
 .../hadoop/hbase/thrift/generated/IOError.java  |    2 +-
 .../hbase/thrift/generated/IllegalArgument.java |    2 +-
 .../hadoop/hbase/thrift/generated/Mutation.java |    2 +-
 .../hadoop/hbase/thrift/generated/TAppend.java  |    2 +-
 .../hadoop/hbase/thrift/generated/TCell.java    |    2 +-
 .../hadoop/hbase/thrift/generated/TColumn.java  |    2 +-
 .../hbase/thrift/generated/TIncrement.java      |    2 +-
 .../hbase/thrift/generated/TRegionInfo.java     |    2 +-
 .../hbase/thrift/generated/TRowResult.java      |    2 +-
 .../hadoop/hbase/thrift/generated/TScan.java    |    2 +-
 .../thrift2/ThriftHBaseServiceHandler.java      |   15 +
 .../hadoop/hbase/thrift2/ThriftUtilities.java   |   15 +
 .../hadoop/hbase/thrift2/generated/TAppend.java |    2 +-
 .../hbase/thrift2/generated/TAuthorization.java |    2 +-
 .../thrift2/generated/TCellVisibility.java      |    2 +-
 .../hadoop/hbase/thrift2/generated/TColumn.java |    2 +-
 .../thrift2/generated/TColumnIncrement.java     |    2 +-
 .../hbase/thrift2/generated/TColumnValue.java   |    2 +-
 .../hadoop/hbase/thrift2/generated/TDelete.java |    2 +-
 .../hadoop/hbase/thrift2/generated/TGet.java    |    2 +-
 .../hbase/thrift2/generated/THBaseService.java  | 1780 +++++++++++++++++-
 .../hbase/thrift2/generated/THRegionInfo.java   |    2 +-
 .../thrift2/generated/THRegionLocation.java     |    2 +-
 .../hbase/thrift2/generated/TIOError.java       |    2 +-
 .../thrift2/generated/TIllegalArgument.java     |    2 +-
 .../hbase/thrift2/generated/TIncrement.java     |    2 +-
 .../hadoop/hbase/thrift2/generated/TPut.java    |    2 +-
 .../hadoop/hbase/thrift2/generated/TResult.java |    2 +-
 .../hbase/thrift2/generated/TRowMutations.java  |    2 +-
 .../hadoop/hbase/thrift2/generated/TScan.java   |    2 +-
 .../hbase/thrift2/generated/TServerName.java    |    2 +-
 .../hbase/thrift2/generated/TTimeRange.java     |    2 +-
 .../apache/hadoop/hbase/thrift2/hbase.thrift    |   45 +
 .../thrift2/TestThriftHBaseServiceHandler.java  |   63 +-
 38 files changed, 1947 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
index 9f11463..6db142e 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class AlreadyExists extends TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
index d89944a..de5dbbc 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A BatchMutation object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
index eaedd61..b8c7b63 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * such as the number of versions, compression settings, etc. It is
  * used as input when creating a table or adding a column.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 dfb49cc..99623af 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
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class Hbase {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
index 3624f6c..1cf37bb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the Hbase master or an Hbase region server.  Also used to return
  * more general Hbase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class IOError extends TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
index 099098e..8dd58c0 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * An IllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
index 5e91281..27f2560 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Mutation object is used to either update or delete a column-value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
index 92a6e2b..fc95104 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TAppend.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * An Append object is used to specify the parameters for performing the append operation.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
index c63939e..eb1448b 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
@@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
  * the timestamp of a cell to a first-class value, making it easy to take
  * note of temporal data. Cell is used all the way from HStore up to HTable.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
index 4e92dc6..2d43593 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds column name and the cell.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
index 10ef656..10c143a 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TIncrement.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * For increments that are not incrementColumnValue
  * equivalents.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
index 760908c..6f2d048 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRegionInfo contains information about an HTable region.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 842e803..9d876a7 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
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Holds row name and then a map of columns to cells.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 1262df0..4e13005 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
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A Scan object is used to specify scanner parameters when opening a scanner.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
index 1343149..9f60d61 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.thrift2;
 
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
+import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deletesFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
+import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
 import org.apache.hadoop.hbase.thrift2.generated.TGet;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
@@ -299,6 +301,19 @@ public class ThriftHBaseServiceHandler implements THBaseService.Iface {
     }
     return Collections.emptyList();
   }
+  
+  @Override
+  public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family,
+      ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations)
+          throws TIOError, TException {
+    try (final Table htable = getTable(table)) {
+      return htable.checkAndMutate(byteBufferToByteArray(row), byteBufferToByteArray(family),
+          byteBufferToByteArray(qualifier), compareOpFromThrift(compareOp),
+          byteBufferToByteArray(value), rowMutationsFromThrift(rowMutations));
+    } catch (IOException e) {
+      throw getTIOError(e);
+    }
+  }
 
   @Override
   public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family,

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 8811e6d..d5cf287 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.ParseFilter;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
+import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
 import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
 import org.apache.hadoop.hbase.thrift2.generated.TDurability;
@@ -530,4 +532,17 @@ public class ThriftUtilities {
       default: return null;
     }
   }
+
+  public static CompareOp compareOpFromThrift(TCompareOp tCompareOp) {
+    switch (tCompareOp.getValue()) {
+      case 0: return CompareOp.LESS;
+      case 1: return CompareOp.LESS_OR_EQUAL;
+      case 2: return CompareOp.EQUAL;
+      case 3: return CompareOp.NOT_EQUAL;
+      case 4: return CompareOp.GREATER_OR_EQUAL;
+      case 5: return CompareOp.GREATER;
+      case 6: return CompareOp.NO_OP;
+      default: return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 ad631ce..17ad5d5 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
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
index d448cdd..a315ab5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TAuthorization.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
index 3ad4f6a..60362bb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TCellVisibility.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
index 0fe7791..f1d95e5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * in a HBase table by column family and optionally
  * a column qualifier and timestamp
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
index 9bdbf97..8ce3af6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and the amount to increment it by
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
index e5cbe56..3ac829d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Represents a single cell and its value.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 5a42d8a..a0efa8e 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
@@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 9fa157c..95c1bce 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
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
  * If you specify a time range and a timestamp the range is ignored.
  * Timestamps on TColumns are ignored.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
 


[19/43] hbase git commit: HBASE-14976 Add RPC call queues to the web ui (Pallavi Adusumilli)

Posted by sy...@apache.org.
HBASE-14976 Add RPC call queues to the web ui (Pallavi Adusumilli)


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

Branch: refs/heads/hbase-12439
Commit: 8c921ea94f330b91d4fd55aa1706fc64b1646de1
Parents: b68542e
Author: Mikhail Antonov <an...@apache.org>
Authored: Sat Dec 19 12:44:30 2015 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Sat Dec 19 12:44:30 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon | 3 ++-
 .../hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon   | 8 +++++++-
 .../java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java | 6 ++++++
 3 files changed, 15 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8c921ea9/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index f38cce9..158a239 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -104,7 +104,8 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 
     <section>
     <h2>Server Metrics</h2>
-    <& ServerMetricsTmpl; mWrap = regionServer.getRegionServerMetrics().getRegionServerWrapper(); &>
+    <& ServerMetricsTmpl; mWrap = regionServer.getRegionServerMetrics().getRegionServerWrapper();
+      mServerWrap = regionServer.getRpcServer().getMetrics().getHBaseServerWrapper(); &>
     </section>
 
     <section>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8c921ea9/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index 13ccc3b..4f6a754 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -18,10 +18,12 @@ limitations under the License.
 </%doc>
 <%args>
 MetricsRegionServerWrapper mWrap;
+MetricsHBaseServerWrapper mServerWrap;
 </%args>
 <%import>
 java.util.*;
 org.apache.hadoop.hbase.regionserver.HRegionServer;
+org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapper;
 org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapper;
 org.apache.hadoop.hbase.util.Bytes;
 org.apache.hadoop.hbase.HRegionInfo;
@@ -61,7 +63,7 @@ java.lang.management.ManagementFactory;
             <& storeStats; mWrap = mWrap &>
         </div>
         <div class="tab-pane" id="tab_queueStats">
-            <& queueStats; mWrap = mWrap  &>
+            <& queueStats; mWrap = mWrap; mServerWrap = mServerWrap; &>
         </div>
     </div>
 </div>
@@ -184,16 +186,20 @@ MetricsRegionServerWrapper mWrap;
 <%def queueStats>
 <%args>
 MetricsRegionServerWrapper mWrap;
+MetricsHBaseServerWrapper mServerWrap;
 </%args>
 <table class="table table-striped">
 <tr>
     <th>Compaction Queue Size</th>
     <th>Flush Queue Size</th>
+    <th>Call Queue Size (bytes)</th>
 
 </tr>
 <tr>
     <td><% mWrap.getCompactionQueueSize() %></td>
     <td><% mWrap.getFlushQueueSize() %></td>
+    <td><% StringUtils.TraditionalBinaryPrefix.long2String(mServerWrap.getTotalQueueSize(),
+      "", 1) %></td>
 </tr>
 </table>
 </%def>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8c921ea9/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
index 05bebb8..e514f5f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServer.java
@@ -32,8 +32,10 @@ import org.apache.hadoop.hbase.exceptions.RegionMovedException;
 @InterfaceAudience.Private
 public class MetricsHBaseServer {
   private MetricsHBaseServerSource source;
+  private MetricsHBaseServerWrapper serverWrapper;
 
   public MetricsHBaseServer(String serverName, MetricsHBaseServerWrapper wrapper) {
+    serverWrapper = wrapper;
     source = CompatibilitySingletonFactory.getInstance(MetricsHBaseServerSourceFactory.class)
                                           .create(serverName, wrapper);
   }
@@ -115,4 +117,8 @@ public class MetricsHBaseServer {
   public MetricsHBaseServerSource getMetricsSource() {
     return source;
   }
+
+  public MetricsHBaseServerWrapper getHBaseServerWrapper() {
+    return serverWrapper;
+  }
 }


[28/43] hbase git commit: HBASE-15028 Minor fix on RegionGroupingProvider (Yu Li)

Posted by sy...@apache.org.
HBASE-15028 Minor fix on RegionGroupingProvider (Yu Li)


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

Branch: refs/heads/hbase-12439
Commit: 2439f7a68843ddff5b468a203e5856705e574599
Parents: 95a13b5
Author: tedyu <yu...@gmail.com>
Authored: Tue Dec 22 06:47:33 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Tue Dec 22 06:47:33 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/2439f7a6/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 2885428..0aeaccf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -185,7 +185,7 @@ class RegionGroupingProvider implements WALProvider {
   }
 
   private WAL getWAL(final String group) throws IOException {
-    WAL log = cached.get(walCacheLock);
+    WAL log = cached.get(group);
     if (null == log) {
       // only lock when need to create wal, and need to lock since
       // creating hlog on fs is time consuming


[04/43] hbase git commit: HBASE-15000 Fix javadoc warn in LoadIncrementalHFiles. (Ashish)

Posted by sy...@apache.org.
HBASE-15000 Fix javadoc warn in LoadIncrementalHFiles. (Ashish)


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

Branch: refs/heads/hbase-12439
Commit: 80fc18d2e1f72d312a4ab90a32d6e44c9b560493
Parents: cf458d3
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Dec 17 22:47:48 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Dec 17 22:47:48 2015 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java    | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/80fc18d2/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 5064c05..0e35427 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -425,11 +425,12 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
    * @param hfilesDir directory containing list of hfiles to be loaded into the table
    * @param table table to which hfiles should be loaded
    * @param queue queue which needs to be loaded into the table
+   * @param validateHFile if true hfiles will be validated for its format
    * @throws IOException If any I/O or network error occurred
    */
-  public void prepareHFileQueue(Path hfofDir, Table table, Deque<LoadQueueItem> queue,
+  public void prepareHFileQueue(Path hfilesDir, Table table, Deque<LoadQueueItem> queue,
       boolean validateHFile) throws IOException {
-    discoverLoadQueue(queue, hfofDir, validateHFile);
+    discoverLoadQueue(queue, hfilesDir, validateHFile);
     validateFamiliesInHFiles(table, queue);
   }
 


[22/43] hbase git commit: HBASE-14976 fix tests

Posted by sy...@apache.org.
HBASE-14976 fix tests

Signed-off-by: Mikhail Antonov <an...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: f56abf730f45823b2b6dd6d2daa5e33c69f3c86b
Parents: a7ba311
Author: Pallavi Adusumilli <pa...@fb.com>
Authored: Mon Dec 21 10:57:25 2015 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Mon Dec 21 11:35:59 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/TestRSStatusServlet.java    | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f56abf73/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index 732df4b..a9115f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -49,6 +49,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
+import org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperStub;
+import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 
 /**
  * Tests for the region server status page and its template.
@@ -58,6 +61,7 @@ public class TestRSStatusServlet {
   private static final Log LOG = LogFactory.getLog(TestRSStatusServlet.class);
   private HRegionServer rs;
   private RSRpcServices rpcServices;
+  private RpcServerInterface rpcServer;
 
   static final int FAKE_IPC_PORT = 1585;
   static final int FAKE_WEB_PORT = 1586;
@@ -74,9 +78,11 @@ public class TestRSStatusServlet {
   public void setupBasicMocks() throws IOException, ServiceException {
     rs = Mockito.mock(HRegionServer.class);
     rpcServices = Mockito.mock(RSRpcServices.class);
+    rpcServer = Mockito.mock(RpcServerInterface.class);
     Mockito.doReturn(HBaseConfiguration.create())
       .when(rs).getConfiguration();
     Mockito.doReturn(rpcServices).when(rs).getRSRpcServices();
+    Mockito.doReturn(rpcServer).when(rs).getRpcServer();
     Mockito.doReturn(fakeResponse).when(rpcServices).getServerInfo(
       (RpcController)Mockito.any(), (GetServerInfoRequest)Mockito.any());
     // Fake ZKW
@@ -98,6 +104,10 @@ public class TestRSStatusServlet {
     MetricsRegionServer rms = Mockito.mock(MetricsRegionServer.class);
     Mockito.doReturn(new MetricsRegionServerWrapperStub()).when(rms).getRegionServerWrapper();
     Mockito.doReturn(rms).when(rs).getRegionServerMetrics();
+
+    MetricsHBaseServer ms = Mockito.mock(MetricsHBaseServer.class);
+    Mockito.doReturn(new MetricsHBaseServerWrapperStub()).when(ms).getHBaseServerWrapper();
+    Mockito.doReturn(ms).when(rpcServer).getMetrics();
   }
   
   @Test


[10/43] hbase git commit: HBASE-11985 Document sizing rules of thumb

Posted by sy...@apache.org.
HBASE-11985 Document sizing rules of thumb


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

Branch: refs/heads/hbase-12439
Commit: 7a4590dfdbda1250f8203e30f6ba1ad0c8094928
Parents: 4bfeccb
Author: Misty Stanley-Jones <ms...@cloudera.com>
Authored: Thu Dec 17 11:29:09 2015 -0800
Committer: Misty Stanley-Jones <ms...@cloudera.com>
Committed: Fri Dec 18 08:34:39 2015 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/schema_design.adoc | 44 +++++++++++++++++++++
 1 file changed, 44 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7a4590df/src/main/asciidoc/_chapters/schema_design.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/schema_design.adoc b/src/main/asciidoc/_chapters/schema_design.adoc
index e5fdd23..5cf8d12 100644
--- a/src/main/asciidoc/_chapters/schema_design.adoc
+++ b/src/main/asciidoc/_chapters/schema_design.adoc
@@ -76,6 +76,50 @@ When changes are made to either Tables or ColumnFamilies (e.g. region size, bloc
 
 See <<store,store>> for more information on StoreFiles.
 
+[[table_schema_rules_of_thumb]]
+== Table Schema Rules Of Thumb
+
+There are many different data sets, with different access patterns and service-level
+expectations. Therefore, these rules of thumb are only an overview. Read the rest
+of this chapter to get more details after you have gone through this list.
+
+* Aim to have regions sized between 10 and 50 GB.
+* Aim to have cells no larger than 10 MB, or 50 MB if you use <<mob>>. Otherwise,
+consider storing your cell data in HDFS and store a pointer to the data in HBase.
+* A typical schema has between 1 and 3 column families per table. HBase tables should
+not be designed to mimic RDBMS tables.
+* Around 50-100 regions is a good number for a table with 1 or 2 column families.
+Remember that a region is a contiguous segment of a column family.
+* Keep your column family names as short as possible. The column family names are
+stored for every value (ignoring prefix encoding). They should not be self-documenting
+and descriptive like in a typical RDBMS.
+* If you are storing time-based machine data or logging information, and the row key
+is based on device ID or service ID plus time, you can end up with a pattern where
+older data regions never have additional writes beyond a certain age. In this type
+of situation, you end up with a small number of active regions and a large number
+of older regions which have no new writes. For these situations, you can tolerate
+a larger number of regions because your resource consumption is driven by the active
+regions only.
+* If only one column family is busy with writes, only that column family accomulates
+memory. Be aware of write patterns when allocating resources.
+
+[[regionserver_sizing_rules_of_thumb]]
+= RegionServer Sizing Rules of Thumb
+
+Lars Hofhansl wrote a great
+link:http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html[blog post]
+about RegionServer memory sizing. The upshot is that you probably need more memory
+than you think you need. He goes into the impact of region size, memstore size, HDFS
+replication factor, and other things to check.
+
+[quote, Lars Hofhansl, http://hadoop-hbase.blogspot.com/2013/01/hbase-region-server-memory-sizing.html]
+____
+Personally I would place the maximum disk space per machine that can be served
+exclusively with HBase around 6T, unless you have a very read-heavy workload.
+In that case the Java heap should be 32GB (20G regions, 128M memstores, the rest
+defaults).
+____
+
 [[number.of.cfs]]
 ==  On the number of column families
 


[31/43] hbase git commit: HBASE-15030 Deadlock in master TableNamespaceManager while running IntegrationTestDDLMasterFailover

Posted by sy...@apache.org.
HBASE-15030 Deadlock in master TableNamespaceManager while running IntegrationTestDDLMasterFailover


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

Branch: refs/heads/hbase-12439
Commit: 8e0854c64be553595b8ed44b9856a3d74ad3005f
Parents: e00a04d
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Wed Dec 23 09:46:18 2015 -0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Wed Dec 23 09:46:18 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/TableNamespaceManager.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8e0854c6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 4b36f59..bbeaf76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -133,7 +133,7 @@ public class TableNamespaceManager {
     return nsTable;
   }
 
-  private synchronized boolean acquireSharedLock() throws IOException {
+  private boolean acquireSharedLock() throws IOException {
     try {
       return rwLock.readLock().tryLock(sharedLockTimeoutMs, TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
@@ -141,11 +141,11 @@ public class TableNamespaceManager {
     }
   }
 
-  public synchronized void releaseSharedLock() {
+  public void releaseSharedLock() {
     rwLock.readLock().unlock();
   }
 
-  public synchronized boolean acquireExclusiveLock() {
+  public boolean acquireExclusiveLock() {
     try {
       return rwLock.writeLock().tryLock(exclusiveLockTimeoutMs, TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
@@ -153,7 +153,7 @@ public class TableNamespaceManager {
     }
   }
 
-  public synchronized void releaseExclusiveLock() {
+  public void releaseExclusiveLock() {
     rwLock.writeLock().unlock();
   }
 


[41/43] hbase git commit: HBASE-14800 Expose checkAndMutate via Thrift2 (Josh Elser)

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
index 98b259b..0e54f13 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class THBaseService {
 
   public interface Iface {
@@ -247,6 +247,29 @@ public class THBaseService {
      */
     public List<THRegionLocation> getAllRegionLocations(ByteBuffer table) throws TIOError, org.apache.thrift.TException;
 
+    /**
+     * Atomically checks if a row/family/qualifier value matches the expected
+     * value. If it does, it mutates the row.
+     * 
+     * @return true if the row was mutated, false otherwise
+     * 
+     * @param table to check in and delete from
+     * 
+     * @param row row to check
+     * 
+     * @param family column family to check
+     * 
+     * @param qualifier column qualifier to check
+     * 
+     * @param compareOp comparison to make on the value
+     * 
+     * @param value the expected value to be compared against, if not provided the
+     * check is for the non-existence of the column in question
+     * 
+     * @param rowMutations row mutations to execute if the value matches
+     */
+    public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations) throws TIOError, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -287,6 +310,8 @@ public class THBaseService {
 
     public void getAllRegionLocations(ByteBuffer table, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -794,6 +819,38 @@ public class THBaseService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getAllRegionLocations failed: unknown result");
     }
 
+    public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations) throws TIOError, org.apache.thrift.TException
+    {
+      send_checkAndMutate(table, row, family, qualifier, compareOp, value, rowMutations);
+      return recv_checkAndMutate();
+    }
+
+    public void send_checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations) throws org.apache.thrift.TException
+    {
+      checkAndMutate_args args = new checkAndMutate_args();
+      args.setTable(table);
+      args.setRow(row);
+      args.setFamily(family);
+      args.setQualifier(qualifier);
+      args.setCompareOp(compareOp);
+      args.setValue(value);
+      args.setRowMutations(rowMutations);
+      sendBase("checkAndMutate", args);
+    }
+
+    public boolean recv_checkAndMutate() throws TIOError, org.apache.thrift.TException
+    {
+      checkAndMutate_result result = new checkAndMutate_result();
+      receiveBase(result, "checkAndMutate");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkAndMutate failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -1466,6 +1523,56 @@ public class THBaseService {
       }
     }
 
+    public void checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      checkAndMutate_call method_call = new checkAndMutate_call(table, row, family, qualifier, compareOp, value, rowMutations, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class checkAndMutate_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ByteBuffer table;
+      private ByteBuffer row;
+      private ByteBuffer family;
+      private ByteBuffer qualifier;
+      private TCompareOp compareOp;
+      private ByteBuffer value;
+      private TRowMutations rowMutations;
+      public checkAndMutate_call(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.table = table;
+        this.row = row;
+        this.family = family;
+        this.qualifier = qualifier;
+        this.compareOp = compareOp;
+        this.value = value;
+        this.rowMutations = rowMutations;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("checkAndMutate", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        checkAndMutate_args args = new checkAndMutate_args();
+        args.setTable(table);
+        args.setRow(row);
+        args.setFamily(family);
+        args.setQualifier(qualifier);
+        args.setCompareOp(compareOp);
+        args.setValue(value);
+        args.setRowMutations(rowMutations);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public boolean getResult() throws TIOError, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_checkAndMutate();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1497,6 +1604,7 @@ public class THBaseService {
       processMap.put("getScannerResults", new getScannerResults());
       processMap.put("getRegionLocation", new getRegionLocation());
       processMap.put("getAllRegionLocations", new getAllRegionLocations());
+      processMap.put("checkAndMutate", new checkAndMutate());
       return processMap;
     }
 
@@ -1940,6 +2048,31 @@ public class THBaseService {
       }
     }
 
+    public static class checkAndMutate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkAndMutate_args> {
+      public checkAndMutate() {
+        super("checkAndMutate");
+      }
+
+      public checkAndMutate_args getEmptyArgsInstance() {
+        return new checkAndMutate_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public checkAndMutate_result getResult(I iface, checkAndMutate_args args) throws org.apache.thrift.TException {
+        checkAndMutate_result result = new checkAndMutate_result();
+        try {
+          result.success = iface.checkAndMutate(args.table, args.row, args.family, args.qualifier, args.compareOp, args.value, args.rowMutations);
+          result.setSuccessIsSet(true);
+        } catch (TIOError io) {
+          result.io = io;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -1971,6 +2104,7 @@ public class THBaseService {
       processMap.put("getScannerResults", new getScannerResults());
       processMap.put("getRegionLocation", new getRegionLocation());
       processMap.put("getAllRegionLocations", new getAllRegionLocations());
+      processMap.put("checkAndMutate", new checkAndMutate());
       return processMap;
     }
 
@@ -3009,6 +3143,64 @@ public class THBaseService {
       }
     }
 
+    public static class checkAndMutate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, checkAndMutate_args, Boolean> {
+      public checkAndMutate() {
+        super("checkAndMutate");
+      }
+
+      public checkAndMutate_args getEmptyArgsInstance() {
+        return new checkAndMutate_args();
+      }
+
+      public AsyncMethodCallback<Boolean> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Boolean>() { 
+          public void onComplete(Boolean o) {
+            checkAndMutate_result result = new checkAndMutate_result();
+            result.success = o;
+            result.setSuccessIsSet(true);
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            checkAndMutate_result result = new checkAndMutate_result();
+            if (e instanceof TIOError) {
+                        result.io = (TIOError) e;
+                        result.setIoIsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, checkAndMutate_args args, org.apache.thrift.async.AsyncMethodCallback<Boolean> resultHandler) throws TException {
+        iface.checkAndMutate(args.table, args.row, args.family, args.qualifier, args.compareOp, args.value, args.rowMutations,resultHandler);
+      }
+    }
+
   }
 
   public static class exists_args implements org.apache.thrift.TBase<exists_args, exists_args._Fields>, java.io.Serializable, Cloneable, Comparable<exists_args>   {
@@ -21434,4 +21626,1590 @@ public class THBaseService {
 
   }
 
+  public static class checkAndMutate_args implements org.apache.thrift.TBase<checkAndMutate_args, checkAndMutate_args._Fields>, java.io.Serializable, Cloneable, Comparable<checkAndMutate_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndMutate_args");
+
+    private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField QUALIFIER_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifier", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField COMPARE_OP_FIELD_DESC = new org.apache.thrift.protocol.TField("compareOp", org.apache.thrift.protocol.TType.I32, (short)5);
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)6);
+    private static final org.apache.thrift.protocol.TField ROW_MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("rowMutations", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new checkAndMutate_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new checkAndMutate_argsTupleSchemeFactory());
+    }
+
+    /**
+     * to check in and delete from
+     */
+    public ByteBuffer table; // required
+    /**
+     * row to check
+     */
+    public ByteBuffer row; // required
+    /**
+     * column family to check
+     */
+    public ByteBuffer family; // required
+    /**
+     * column qualifier to check
+     */
+    public ByteBuffer qualifier; // required
+    /**
+     * comparison to make on the value
+     * 
+     * @see TCompareOp
+     */
+    public TCompareOp compareOp; // required
+    /**
+     * the expected value to be compared against, if not provided the
+     * check is for the non-existence of the column in question
+     */
+    public ByteBuffer value; // required
+    /**
+     * row mutations to execute if the value matches
+     */
+    public TRowMutations rowMutations; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      /**
+       * to check in and delete from
+       */
+      TABLE((short)1, "table"),
+      /**
+       * row to check
+       */
+      ROW((short)2, "row"),
+      /**
+       * column family to check
+       */
+      FAMILY((short)3, "family"),
+      /**
+       * column qualifier to check
+       */
+      QUALIFIER((short)4, "qualifier"),
+      /**
+       * comparison to make on the value
+       * 
+       * @see TCompareOp
+       */
+      COMPARE_OP((short)5, "compareOp"),
+      /**
+       * the expected value to be compared against, if not provided the
+       * check is for the non-existence of the column in question
+       */
+      VALUE((short)6, "value"),
+      /**
+       * row mutations to execute if the value matches
+       */
+      ROW_MUTATIONS((short)7, "rowMutations");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TABLE
+            return TABLE;
+          case 2: // ROW
+            return ROW;
+          case 3: // FAMILY
+            return FAMILY;
+          case 4: // QUALIFIER
+            return QUALIFIER;
+          case 5: // COMPARE_OP
+            return COMPARE_OP;
+          case 6: // VALUE
+            return VALUE;
+          case 7: // ROW_MUTATIONS
+            return ROW_MUTATIONS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    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);
+      tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.ROW, new org.apache.thrift.meta_data.FieldMetaData("row", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.FAMILY, new org.apache.thrift.meta_data.FieldMetaData("family", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.QUALIFIER, new org.apache.thrift.meta_data.FieldMetaData("qualifier", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.COMPARE_OP, new org.apache.thrift.meta_data.FieldMetaData("compareOp", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TCompareOp.class)));
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
+      tmpMap.put(_Fields.ROW_MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("rowMutations", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowMutations.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkAndMutate_args.class, metaDataMap);
+    }
+
+    public checkAndMutate_args() {
+    }
+
+    public checkAndMutate_args(
+      ByteBuffer table,
+      ByteBuffer row,
+      ByteBuffer family,
+      ByteBuffer qualifier,
+      TCompareOp compareOp,
+      ByteBuffer value,
+      TRowMutations rowMutations)
+    {
+      this();
+      this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
+      this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
+      this.family = org.apache.thrift.TBaseHelper.copyBinary(family);
+      this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+      this.compareOp = compareOp;
+      this.value = org.apache.thrift.TBaseHelper.copyBinary(value);
+      this.rowMutations = rowMutations;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public checkAndMutate_args(checkAndMutate_args other) {
+      if (other.isSetTable()) {
+        this.table = org.apache.thrift.TBaseHelper.copyBinary(other.table);
+      }
+      if (other.isSetRow()) {
+        this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
+      }
+      if (other.isSetFamily()) {
+        this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family);
+      }
+      if (other.isSetQualifier()) {
+        this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier);
+      }
+      if (other.isSetCompareOp()) {
+        this.compareOp = other.compareOp;
+      }
+      if (other.isSetValue()) {
+        this.value = org.apache.thrift.TBaseHelper.copyBinary(other.value);
+      }
+      if (other.isSetRowMutations()) {
+        this.rowMutations = new TRowMutations(other.rowMutations);
+      }
+    }
+
+    public checkAndMutate_args deepCopy() {
+      return new checkAndMutate_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.table = null;
+      this.row = null;
+      this.family = null;
+      this.qualifier = null;
+      this.compareOp = null;
+      this.value = null;
+      this.rowMutations = null;
+    }
+
+    /**
+     * to check in and delete from
+     */
+    public byte[] getTable() {
+      setTable(org.apache.thrift.TBaseHelper.rightSize(table));
+      return table == null ? null : table.array();
+    }
+
+    public ByteBuffer bufferForTable() {
+      return org.apache.thrift.TBaseHelper.copyBinary(table);
+    }
+
+    /**
+     * to check in and delete from
+     */
+    public checkAndMutate_args setTable(byte[] table) {
+      this.table = table == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(table, table.length));
+      return this;
+    }
+
+    public checkAndMutate_args setTable(ByteBuffer table) {
+      this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
+      return this;
+    }
+
+    public void unsetTable() {
+      this.table = null;
+    }
+
+    /** Returns true if field table is set (has been assigned a value) and false otherwise */
+    public boolean isSetTable() {
+      return this.table != null;
+    }
+
+    public void setTableIsSet(boolean value) {
+      if (!value) {
+        this.table = null;
+      }
+    }
+
+    /**
+     * row to check
+     */
+    public byte[] getRow() {
+      setRow(org.apache.thrift.TBaseHelper.rightSize(row));
+      return row == null ? null : row.array();
+    }
+
+    public ByteBuffer bufferForRow() {
+      return org.apache.thrift.TBaseHelper.copyBinary(row);
+    }
+
+    /**
+     * row to check
+     */
+    public checkAndMutate_args setRow(byte[] row) {
+      this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
+      return this;
+    }
+
+    public checkAndMutate_args setRow(ByteBuffer row) {
+      this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
+      return this;
+    }
+
+    public void unsetRow() {
+      this.row = null;
+    }
+
+    /** Returns true if field row is set (has been assigned a value) and false otherwise */
+    public boolean isSetRow() {
+      return this.row != null;
+    }
+
+    public void setRowIsSet(boolean value) {
+      if (!value) {
+        this.row = null;
+      }
+    }
+
+    /**
+     * column family to check
+     */
+    public byte[] getFamily() {
+      setFamily(org.apache.thrift.TBaseHelper.rightSize(family));
+      return family == null ? null : family.array();
+    }
+
+    public ByteBuffer bufferForFamily() {
+      return org.apache.thrift.TBaseHelper.copyBinary(family);
+    }
+
+    /**
+     * column family to check
+     */
+    public checkAndMutate_args setFamily(byte[] family) {
+      this.family = family == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(family, family.length));
+      return this;
+    }
+
+    public checkAndMutate_args setFamily(ByteBuffer family) {
+      this.family = org.apache.thrift.TBaseHelper.copyBinary(family);
+      return this;
+    }
+
+    public void unsetFamily() {
+      this.family = null;
+    }
+
+    /** Returns true if field family is set (has been assigned a value) and false otherwise */
+    public boolean isSetFamily() {
+      return this.family != null;
+    }
+
+    public void setFamilyIsSet(boolean value) {
+      if (!value) {
+        this.family = null;
+      }
+    }
+
+    /**
+     * column qualifier to check
+     */
+    public byte[] getQualifier() {
+      setQualifier(org.apache.thrift.TBaseHelper.rightSize(qualifier));
+      return qualifier == null ? null : qualifier.array();
+    }
+
+    public ByteBuffer bufferForQualifier() {
+      return org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+    }
+
+    /**
+     * column qualifier to check
+     */
+    public checkAndMutate_args setQualifier(byte[] qualifier) {
+      this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
+      return this;
+    }
+
+    public checkAndMutate_args setQualifier(ByteBuffer qualifier) {
+      this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
+      return this;
+    }
+
+    public void unsetQualifier() {
+      this.qualifier = null;
+    }
+
+    /** Returns true if field qualifier is set (has been assigned a value) and false otherwise */
+    public boolean isSetQualifier() {
+      return this.qualifier != null;
+    }
+
+    public void setQualifierIsSet(boolean value) {
+      if (!value) {
+        this.qualifier = null;
+      }
+    }
+
+    /**
+     * comparison to make on the value
+     * 
+     * @see TCompareOp
+     */
+    public TCompareOp getCompareOp() {
+      return this.compareOp;
+    }
+
+    /**
+     * comparison to make on the value
+     * 
+     * @see TCompareOp
+     */
+    public checkAndMutate_args setCompareOp(TCompareOp compareOp) {
+      this.compareOp = compareOp;
+      return this;
+    }
+
+    public void unsetCompareOp() {
+      this.compareOp = null;
+    }
+
+    /** Returns true if field compareOp is set (has been assigned a value) and false otherwise */
+    public boolean isSetCompareOp() {
+      return this.compareOp != null;
+    }
+
+    public void setCompareOpIsSet(boolean value) {
+      if (!value) {
+        this.compareOp = null;
+      }
+    }
+
+    /**
+     * the expected value to be compared against, if not provided the
+     * check is for the non-existence of the column in question
+     */
+    public byte[] getValue() {
+      setValue(org.apache.thrift.TBaseHelper.rightSize(value));
+      return value == null ? null : value.array();
+    }
+
+    public ByteBuffer bufferForValue() {
+      return org.apache.thrift.TBaseHelper.copyBinary(value);
+    }
+
+    /**
+     * the expected value to be compared against, if not provided the
+     * check is for the non-existence of the column in question
+     */
+    public checkAndMutate_args setValue(byte[] value) {
+      this.value = value == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(value, value.length));
+      return this;
+    }
+
+    public checkAndMutate_args setValue(ByteBuffer value) {
+      this.value = org.apache.thrift.TBaseHelper.copyBinary(value);
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    /**
+     * row mutations to execute if the value matches
+     */
+    public TRowMutations getRowMutations() {
+      return this.rowMutations;
+    }
+
+    /**
+     * row mutations to execute if the value matches
+     */
+    public checkAndMutate_args setRowMutations(TRowMutations rowMutations) {
+      this.rowMutations = rowMutations;
+      return this;
+    }
+
+    public void unsetRowMutations() {
+      this.rowMutations = null;
+    }
+
+    /** Returns true if field rowMutations is set (has been assigned a value) and false otherwise */
+    public boolean isSetRowMutations() {
+      return this.rowMutations != null;
+    }
+
+    public void setRowMutationsIsSet(boolean value) {
+      if (!value) {
+        this.rowMutations = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TABLE:
+        if (value == null) {
+          unsetTable();
+        } else {
+          setTable((ByteBuffer)value);
+        }
+        break;
+
+      case ROW:
+        if (value == null) {
+          unsetRow();
+        } else {
+          setRow((ByteBuffer)value);
+        }
+        break;
+
+      case FAMILY:
+        if (value == null) {
+          unsetFamily();
+        } else {
+          setFamily((ByteBuffer)value);
+        }
+        break;
+
+      case QUALIFIER:
+        if (value == null) {
+          unsetQualifier();
+        } else {
+          setQualifier((ByteBuffer)value);
+        }
+        break;
+
+      case COMPARE_OP:
+        if (value == null) {
+          unsetCompareOp();
+        } else {
+          setCompareOp((TCompareOp)value);
+        }
+        break;
+
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((ByteBuffer)value);
+        }
+        break;
+
+      case ROW_MUTATIONS:
+        if (value == null) {
+          unsetRowMutations();
+        } else {
+          setRowMutations((TRowMutations)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TABLE:
+        return getTable();
+
+      case ROW:
+        return getRow();
+
+      case FAMILY:
+        return getFamily();
+
+      case QUALIFIER:
+        return getQualifier();
+
+      case COMPARE_OP:
+        return getCompareOp();
+
+      case VALUE:
+        return getValue();
+
+      case ROW_MUTATIONS:
+        return getRowMutations();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TABLE:
+        return isSetTable();
+      case ROW:
+        return isSetRow();
+      case FAMILY:
+        return isSetFamily();
+      case QUALIFIER:
+        return isSetQualifier();
+      case COMPARE_OP:
+        return isSetCompareOp();
+      case VALUE:
+        return isSetValue();
+      case ROW_MUTATIONS:
+        return isSetRowMutations();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof checkAndMutate_args)
+        return this.equals((checkAndMutate_args)that);
+      return false;
+    }
+
+    public boolean equals(checkAndMutate_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_table = true && this.isSetTable();
+      boolean that_present_table = true && that.isSetTable();
+      if (this_present_table || that_present_table) {
+        if (!(this_present_table && that_present_table))
+          return false;
+        if (!this.table.equals(that.table))
+          return false;
+      }
+
+      boolean this_present_row = true && this.isSetRow();
+      boolean that_present_row = true && that.isSetRow();
+      if (this_present_row || that_present_row) {
+        if (!(this_present_row && that_present_row))
+          return false;
+        if (!this.row.equals(that.row))
+          return false;
+      }
+
+      boolean this_present_family = true && this.isSetFamily();
+      boolean that_present_family = true && that.isSetFamily();
+      if (this_present_family || that_present_family) {
+        if (!(this_present_family && that_present_family))
+          return false;
+        if (!this.family.equals(that.family))
+          return false;
+      }
+
+      boolean this_present_qualifier = true && this.isSetQualifier();
+      boolean that_present_qualifier = true && that.isSetQualifier();
+      if (this_present_qualifier || that_present_qualifier) {
+        if (!(this_present_qualifier && that_present_qualifier))
+          return false;
+        if (!this.qualifier.equals(that.qualifier))
+          return false;
+      }
+
+      boolean this_present_compareOp = true && this.isSetCompareOp();
+      boolean that_present_compareOp = true && that.isSetCompareOp();
+      if (this_present_compareOp || that_present_compareOp) {
+        if (!(this_present_compareOp && that_present_compareOp))
+          return false;
+        if (!this.compareOp.equals(that.compareOp))
+          return false;
+      }
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      boolean this_present_rowMutations = true && this.isSetRowMutations();
+      boolean that_present_rowMutations = true && that.isSetRowMutations();
+      if (this_present_rowMutations || that_present_rowMutations) {
+        if (!(this_present_rowMutations && that_present_rowMutations))
+          return false;
+        if (!this.rowMutations.equals(that.rowMutations))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_table = true && (isSetTable());
+      list.add(present_table);
+      if (present_table)
+        list.add(table);
+
+      boolean present_row = true && (isSetRow());
+      list.add(present_row);
+      if (present_row)
+        list.add(row);
+
+      boolean present_family = true && (isSetFamily());
+      list.add(present_family);
+      if (present_family)
+        list.add(family);
+
+      boolean present_qualifier = true && (isSetQualifier());
+      list.add(present_qualifier);
+      if (present_qualifier)
+        list.add(qualifier);
+
+      boolean present_compareOp = true && (isSetCompareOp());
+      list.add(present_compareOp);
+      if (present_compareOp)
+        list.add(compareOp.getValue());
+
+      boolean present_value = true && (isSetValue());
+      list.add(present_value);
+      if (present_value)
+        list.add(value);
+
+      boolean present_rowMutations = true && (isSetRowMutations());
+      list.add(present_rowMutations);
+      if (present_rowMutations)
+        list.add(rowMutations);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(checkAndMutate_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTable()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRow()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetFamily()).compareTo(other.isSetFamily());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetFamily()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, other.family);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(other.isSetQualifier());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetQualifier()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, other.qualifier);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCompareOp()).compareTo(other.isSetCompareOp());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCompareOp()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compareOp, other.compareOp);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetRowMutations()).compareTo(other.isSetRowMutations());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRowMutations()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowMutations, other.rowMutations);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("checkAndMutate_args(");
+      boolean first = true;
+
+      sb.append("table:");
+      if (this.table == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.table, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("row:");
+      if (this.row == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.row, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("family:");
+      if (this.family == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.family, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("qualifier:");
+      if (this.qualifier == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.qualifier, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("compareOp:");
+      if (this.compareOp == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.compareOp);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.value, sb);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("rowMutations:");
+      if (this.rowMutations == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.rowMutations);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      if (table == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'table' was not present! Struct: " + toString());
+      }
+      if (row == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'row' was not present! Struct: " + toString());
+      }
+      if (family == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'family' was not present! Struct: " + toString());
+      }
+      if (qualifier == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifier' was not present! Struct: " + toString());
+      }
+      if (compareOp == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'compareOp' was not present! Struct: " + toString());
+      }
+      if (rowMutations == null) {
+        throw new org.apache.thrift.protocol.TProtocolException("Required field 'rowMutations' was not present! Struct: " + toString());
+      }
+      // check for sub-struct validity
+      if (rowMutations != null) {
+        rowMutations.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class checkAndMutate_argsStandardSchemeFactory implements SchemeFactory {
+      public checkAndMutate_argsStandardScheme getScheme() {
+        return new checkAndMutate_argsStandardScheme();
+      }
+    }
+
+    private static class checkAndMutate_argsStandardScheme extends StandardScheme<checkAndMutate_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, checkAndMutate_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TABLE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.table = iprot.readBinary();
+                struct.setTableIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // ROW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.row = iprot.readBinary();
+                struct.setRowIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // FAMILY
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.family = iprot.readBinary();
+                struct.setFamilyIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // QUALIFIER
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.qualifier = iprot.readBinary();
+                struct.setQualifierIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // COMPARE_OP
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.compareOp = org.apache.hadoop.hbase.thrift2.generated.TCompareOp.findByValue(iprot.readI32());
+                struct.setCompareOpIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 6: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readBinary();
+                struct.setValueIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 7: // ROW_MUTATIONS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.rowMutations = new TRowMutations();
+                struct.rowMutations.read(iprot);
+                struct.setRowMutationsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, checkAndMutate_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.table != null) {
+          oprot.writeFieldBegin(TABLE_FIELD_DESC);
+          oprot.writeBinary(struct.table);
+          oprot.writeFieldEnd();
+        }
+        if (struct.row != null) {
+          oprot.writeFieldBegin(ROW_FIELD_DESC);
+          oprot.writeBinary(struct.row);
+          oprot.writeFieldEnd();
+        }
+        if (struct.family != null) {
+          oprot.writeFieldBegin(FAMILY_FIELD_DESC);
+          oprot.writeBinary(struct.family);
+          oprot.writeFieldEnd();
+        }
+        if (struct.qualifier != null) {
+          oprot.writeFieldBegin(QUALIFIER_FIELD_DESC);
+          oprot.writeBinary(struct.qualifier);
+          oprot.writeFieldEnd();
+        }
+        if (struct.compareOp != null) {
+          oprot.writeFieldBegin(COMPARE_OP_FIELD_DESC);
+          oprot.writeI32(struct.compareOp.getValue());
+          oprot.writeFieldEnd();
+        }
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeBinary(struct.value);
+          oprot.writeFieldEnd();
+        }
+        if (struct.rowMutations != null) {
+          oprot.writeFieldBegin(ROW_MUTATIONS_FIELD_DESC);
+          struct.rowMutations.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class checkAndMutate_argsTupleSchemeFactory implements SchemeFactory {
+      public checkAndMutate_argsTupleScheme getScheme() {
+        return new checkAndMutate_argsTupleScheme();
+      }
+    }
+
+    private static class checkAndMutate_argsTupleScheme extends TupleScheme<checkAndMutate_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, checkAndMutate_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        oprot.writeBinary(struct.table);
+        oprot.writeBinary(struct.row);
+        oprot.writeBinary(struct.family);
+        oprot.writeBinary(struct.qualifier);
+        oprot.writeI32(struct.compareOp.getValue());
+        struct.rowMutations.write(oprot);
+        BitSet optionals = new BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeBinary(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, checkAndMutate_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        struct.table = iprot.readBinary();
+        struct.setTableIsSet(true);
+        struct.row = iprot.readBinary();
+        struct.setRowIsSet(true);
+        struct.family = iprot.readBinary();
+        struct.setFamilyIsSet(true);
+        struct.qualifier = iprot.readBinary();
+        struct.setQualifierIsSet(true);
+        struct.compareOp = org.apache.hadoop.hbase.thrift2.generated.TCompareOp.findByValue(iprot.readI32());
+        struct.setCompareOpIsSet(true);
+        struct.rowMutations = new TRowMutations();
+        struct.rowMutations.read(iprot);
+        struct.setRowMutationsIsSet(true);
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readBinary();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class checkAndMutate_result implements org.apache.thrift.TBase<checkAndMutate_result, checkAndMutate_result._Fields>, java.io.Serializable, Cloneable, Comparable<checkAndMutate_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndMutate_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+    private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new checkAndMutate_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new checkAndMutate_resultTupleSchemeFactory());
+    }
+
+    public boolean success; // required
+    public TIOError io; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      IO((short)1, "io");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // IO
+            return IO;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    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);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      tmpMap.put(_Fields.IO, new org.apache.thrift.meta_data.FieldMetaData("io", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkAndMutate_result.class, metaDataMap);
+    }
+
+    public checkAndMutate_result() {
+    }
+
+    public checkAndMutate_result(
+      boolean success,
+      TIOError io)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+      this.io = io;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public checkAndMutate_result(checkAndMutate_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+      if (other.isSetIo()) {
+        this.io = new TIOError(other.io);
+      }
+    }
+
+    public checkAndMutate_result deepCopy() {
+      return new checkAndMutate_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+      this.io = null;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public checkAndMutate_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    public TIOError getIo() {
+      return this.io;
+    }
+
+    public checkAndMutate_result setIo(TIOError io) {
+      this.io = io;
+      return this;
+    }
+
+    public void unsetIo() {
+      this.io = null;
+    }
+
+    /** Returns true if field io is set (has been assigned a value) and false otherwise */
+    public boolean isSetIo() {
+      return this.io != null;
+    }
+
+    public void setIoIsSet(boolean value) {
+      if (!value) {
+        this.io = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Boolean)value);
+        }
+        break;
+
+      case IO:
+        if (value == null) {
+          unsetIo();
+        } else {
+          setIo((TIOError)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return isSuccess();
+
+      case IO:
+        return getIo();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case IO:
+        return isSetIo();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof checkAndMutate_result)
+        return this.equals((checkAndMutate_result)that);
+      return false;
+    }
+
+    public boolean equals(checkAndMutate_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_io = true && this.isSetIo();
+      boolean that_present_io = true && that.isSetIo();
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true;
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_io = true && (isSetIo());
+      list.add(present_io);
+      if (present_io)
+        list.add(io);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(checkAndMutate_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetIo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("checkAndMutate_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("io:");
+      if (this.io == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.io);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class checkAndMutate_resultStandardSchemeFactory implements SchemeFactory {
+      public checkAndMutate_resultStandardScheme getScheme() {
+        return new checkAndMutate_resultStandardScheme();
+      }
+    }
+
+    private static class checkAndMutate_resultStandardScheme extends StandardScheme<checkAndMutate_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, checkAndMutate_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // IO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.io = new TIOError();
+                struct.io.read(iprot);
+                struct.setIoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, checkAndMutate_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.io != null) {
+          oprot.writeFieldBegin(IO_FIELD_DESC);
+          struct.io.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class checkAndMutate_resultTupleSchemeFactory implements SchemeFactory {
+      public checkAndMutate_resultTupleScheme getScheme() {
+        return new checkAndMutate_resultTupleScheme();
+      }
+    }
+
+    private static class checkAndMutate_resultTupleScheme extends TupleScheme<checkAndMutate_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, checkAndMutate_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetIo()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+        if (struct.isSetIo()) {
+          struct.io.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, checkAndMutate_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.io = new TIOError();
+          struct.io.read(iprot);
+          struct.setIoIsSet(true);
+        }
+      }
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
index be6e56d..043548c 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionInfo.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
index 5b51b03..84f1e84 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THRegionLocation.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
index ee41eb5..baf0b3d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
@@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
  * to the HBase master or a HBase region server. Also used to return
  * more general HBase error conditions.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TIOError extends TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
index 4cbb4e6..c2d67bb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * A TIllegalArgument exception indicates an illegal or invalid
  * argument was passed into a procedure.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TIllegalArgument extends TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 36abb52..d21b451 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
@@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 392e402..5a55e03 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
@@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
  * by changing the durability. If you don't provide durability, it defaults to
  * column family's default setting for durability.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
index 962d85f..20c8a90 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * if no Result is found, row and columnValues will not be set.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
index b30e97a..33c46bd 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 /**
  * A TRowMutations object is used to apply a number of Mutations to a single row.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 f4c4d77..4ae759c 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
@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * Any timestamps in the columns are ignored, use timeRange to select by timestamp.
  * Max versions defaults to 1.
  */
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
index 0682675..a25f2c5 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TServerName.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
index 63c91f5..24e5d58 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-11-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2015-12-13")
 public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/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 4ad4cdd..8afeef1 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
@@ -263,6 +263,21 @@ struct THRegionLocation {
   2: required THRegionInfo regionInfo
 }
 
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.filter.CompareFilter$CompareOp.
+ */
+enum TCompareOp {
+  LESS = 0,
+  LESS_OR_EQUAL = 1,
+  EQUAL = 2,
+  NOT_EQUAL = 3,
+  GREATER_OR_EQUAL = 4,
+  GREATER = 5,
+  NO_OP = 6
+}
+
+
 //
 // Exceptions
 //
@@ -556,4 +571,34 @@ service THBaseService {
   ) throws (
     1: TIOError io
   )
+
+  /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it mutates the row.
+   *
+   * @return true if the row was mutated, false otherwise
+   */
+  bool checkAndMutate(
+    /** to check in and delete from */
+    1: required binary table,
+
+    /** row to check */
+    2: required binary row,
+
+    /** column family to check */
+    3: required binary family,
+
+    /** column qualifier to check */
+    4: required binary qualifier,
+
+    /** comparison to make on the value */
+    5: required TCompareOp compareOp,
+
+    /** the expected value to be compared against, if not provided the
+        check is for the non-existence of the column in question */
+    6: binary value,
+
+    /** row mutations to execute if the value matches */
+    7: required TRowMutations rowMutations
+  ) throws (1: TIOError io)
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b8e2e4a1/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 654324d..6fca282 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
+import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
 import org.apache.hadoop.hbase.thrift2.generated.TDelete;
 import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
 import org.apache.hadoop.hbase.thrift2.generated.TGet;
@@ -67,6 +68,7 @@ import org.junit.experimental.categories.Category;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -125,12 +127,16 @@ public class TestThriftHBaseServiceHandler {
     for (int i = 0; i < columnValuesA.size(); i++) {
       TColumnValue a = columnValuesA.get(i);
       TColumnValue b = columnValuesB.get(i);
-      assertArrayEquals(a.getFamily(), b.getFamily());
-      assertArrayEquals(a.getQualifier(), b.getQualifier());
-      assertArrayEquals(a.getValue(), b.getValue());
+      assertTColumnValueEqual(a, b);
     }
   }
 
+  public void assertTColumnValueEqual(TColumnValue a, TColumnValue b) {
+    assertArrayEquals(a.getFamily(), b.getFamily());
+    assertArrayEquals(a.getQualifier(), b.getQualifier());
+    assertArrayEquals(a.getValue(), b.getValue());
+  }
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     UTIL.startMiniCluster();
@@ -1087,5 +1093,56 @@ public class TestThriftHBaseServiceHandler {
     increment = incrementFromThrift(tIncrement);
     assertEquals(increment.getDurability(), Durability.FSYNC_WAL);
   }
+
+  @Test
+  public void testCheckAndMutate() throws Exception {
+    ThriftHBaseServiceHandler handler = createHandler();
+    ByteBuffer table = wrap(tableAname);
+    ByteBuffer row = wrap("row".getBytes());
+    ByteBuffer family = wrap(familyAname);
+    ByteBuffer qualifier = wrap(qualifierAname);
+    ByteBuffer value = wrap(valueAname);
+
+    // Create a mutation to write to 'B', our "mutate" of "checkAndMutate"
+    List<TColumnValue> columnValuesB = new ArrayList<TColumnValue>();
+    TColumnValue columnValueB = new TColumnValue(family, wrap(qualifierBname), wrap(valueBname));
+    columnValuesB.add(columnValueB);
+    TPut putB = new TPut(row, columnValuesB);
+    putB.setColumnValues(columnValuesB);
+
+    TRowMutations tRowMutations = new TRowMutations(row,
+        Arrays.<TMutation> asList(TMutation.put(putB)));
+
+    // Empty table when we begin
+    TResult result = handler.get(table, new TGet(row));
+    assertEquals(0, result.getColumnValuesSize());
+
+    // checkAndMutate -- condition should fail because the value doesn't exist.
+    assertFalse("Expected condition to not pass",
+        handler.checkAndMutate(table, row, family, qualifier, TCompareOp.EQUAL, value,
+            tRowMutations));
+
+    List<TColumnValue> columnValuesA = new ArrayList<TColumnValue>();
+    TColumnValue columnValueA = new TColumnValue(family, qualifier, value);
+    columnValuesA.add(columnValueA);
+
+    // Put an update 'A'
+    handler.put(table, new TPut(row, columnValuesA));
+
+    // Verify that the update is there
+    result = handler.get(table, new TGet(row));
+    assertEquals(1, result.getColumnValuesSize());
+    assertTColumnValueEqual(columnValueA, result.getColumnValues().get(0));
+
+    // checkAndMutate -- condition should pass since we added the value
+    assertTrue("Expected condition to pass",
+        handler.checkAndMutate(table, row, family, qualifier, TCompareOp.EQUAL, value,
+            tRowMutations));
+
+    result = handler.get(table, new TGet(row));
+    assertEquals(2, result.getColumnValuesSize());
+    assertTColumnValueEqual(columnValueA, result.getColumnValues().get(0));
+    assertTColumnValueEqual(columnValueB, result.getColumnValues().get(1));
+  }
 }
 


[08/43] hbase git commit: HBASE-13158 When client supports CellBlock, return the result Cells as controller payload for get(Get) API also.

Posted by sy...@apache.org.
HBASE-13158 When client supports CellBlock, return the result Cells as controller payload for get(Get) API also.


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

Branch: refs/heads/hbase-12439
Commit: 408666a06d1b2da36e650a721cdc26f0db40de1c
Parents: 2716095
Author: anoopsjohn <an...@gmail.com>
Authored: Fri Dec 18 09:50:53 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Fri Dec 18 09:50:53 2015 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/HTable.java  |   2 +-
 .../RpcRetryingCallerWithReadReplicas.java      |   2 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   8 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 209 +++++++++++++++++--
 hbase-protocol/src/main/protobuf/HBase.proto    |   2 +
 .../hadoop/hbase/client/VersionInfoUtil.java    |   8 +
 .../hbase/regionserver/RSRpcServices.java       |  12 +-
 7 files changed, 225 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 51a95e4..d3f3bc4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -438,7 +438,7 @@ public class HTable implements HTableInterface {
           try {
             ClientProtos.GetResponse response = getStub().get(controller, request);
             if (response == null) return null;
-            return ProtobufUtil.toResult(response.getResult());
+            return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
           } catch (ServiceException se) {
             throw ProtobufUtil.getRemoteException(se);
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
index d610d8c..f4e2614 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java
@@ -161,7 +161,7 @@ public class RpcRetryingCallerWithReadReplicas {
         if (response == null) {
           return null;
         }
-        return ProtobufUtil.toResult(response.getResult());
+        return ProtobufUtil.toResult(response.getResult(), controller.cellScanner());
       } catch (ServiceException se) {
         throw ProtobufUtil.getRemoteException(se);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index dd42fd5..b2d5994 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -3137,7 +3137,13 @@ public final class ProtobufUtil {
    */
   public static HBaseProtos.VersionInfo getVersionInfo() {
     HBaseProtos.VersionInfo.Builder builder = HBaseProtos.VersionInfo.newBuilder();
-    builder.setVersion(VersionInfo.getVersion());
+    String version = VersionInfo.getVersion();
+    builder.setVersion(version);
+    String[] components = version.split("\\.");
+    if (components != null && components.length > 2) {
+      builder.setVersionMajor(Integer.parseInt(components[0]));
+      builder.setVersionMinor(Integer.parseInt(components[1]));
+    }
     builder.setUrl(VersionInfo.getUrl());
     builder.setRevision(VersionInfo.getRevision());
     builder.setUser(VersionInfo.getUser());

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
index d9d7956..45dfd09 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
@@ -17962,6 +17962,26 @@ public final class HBaseProtos {
      */
     com.google.protobuf.ByteString
         getSrcChecksumBytes();
+
+    // optional uint32 version_major = 7;
+    /**
+     * <code>optional uint32 version_major = 7;</code>
+     */
+    boolean hasVersionMajor();
+    /**
+     * <code>optional uint32 version_major = 7;</code>
+     */
+    int getVersionMajor();
+
+    // optional uint32 version_minor = 8;
+    /**
+     * <code>optional uint32 version_minor = 8;</code>
+     */
+    boolean hasVersionMinor();
+    /**
+     * <code>optional uint32 version_minor = 8;</code>
+     */
+    int getVersionMinor();
   }
   /**
    * Protobuf type {@code hbase.pb.VersionInfo}
@@ -18048,6 +18068,16 @@ public final class HBaseProtos {
               srcChecksum_ = input.readBytes();
               break;
             }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              versionMajor_ = input.readUInt32();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              versionMinor_ = input.readUInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -18346,6 +18376,38 @@ public final class HBaseProtos {
       }
     }
 
+    // optional uint32 version_major = 7;
+    public static final int VERSION_MAJOR_FIELD_NUMBER = 7;
+    private int versionMajor_;
+    /**
+     * <code>optional uint32 version_major = 7;</code>
+     */
+    public boolean hasVersionMajor() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional uint32 version_major = 7;</code>
+     */
+    public int getVersionMajor() {
+      return versionMajor_;
+    }
+
+    // optional uint32 version_minor = 8;
+    public static final int VERSION_MINOR_FIELD_NUMBER = 8;
+    private int versionMinor_;
+    /**
+     * <code>optional uint32 version_minor = 8;</code>
+     */
+    public boolean hasVersionMinor() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional uint32 version_minor = 8;</code>
+     */
+    public int getVersionMinor() {
+      return versionMinor_;
+    }
+
     private void initFields() {
       version_ = "";
       url_ = "";
@@ -18353,6 +18415,8 @@ public final class HBaseProtos {
       user_ = "";
       date_ = "";
       srcChecksum_ = "";
+      versionMajor_ = 0;
+      versionMinor_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -18408,6 +18472,12 @@ public final class HBaseProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeBytes(6, getSrcChecksumBytes());
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt32(7, versionMajor_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeUInt32(8, versionMinor_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -18441,6 +18511,14 @@ public final class HBaseProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(6, getSrcChecksumBytes());
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(7, versionMajor_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(8, versionMinor_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -18494,6 +18572,16 @@ public final class HBaseProtos {
         result = result && getSrcChecksum()
             .equals(other.getSrcChecksum());
       }
+      result = result && (hasVersionMajor() == other.hasVersionMajor());
+      if (hasVersionMajor()) {
+        result = result && (getVersionMajor()
+            == other.getVersionMajor());
+      }
+      result = result && (hasVersionMinor() == other.hasVersionMinor());
+      if (hasVersionMinor()) {
+        result = result && (getVersionMinor()
+            == other.getVersionMinor());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -18531,6 +18619,14 @@ public final class HBaseProtos {
         hash = (37 * hash) + SRC_CHECKSUM_FIELD_NUMBER;
         hash = (53 * hash) + getSrcChecksum().hashCode();
       }
+      if (hasVersionMajor()) {
+        hash = (37 * hash) + VERSION_MAJOR_FIELD_NUMBER;
+        hash = (53 * hash) + getVersionMajor();
+      }
+      if (hasVersionMinor()) {
+        hash = (37 * hash) + VERSION_MINOR_FIELD_NUMBER;
+        hash = (53 * hash) + getVersionMinor();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -18656,6 +18752,10 @@ public final class HBaseProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         srcChecksum_ = "";
         bitField0_ = (bitField0_ & ~0x00000020);
+        versionMajor_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        versionMinor_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -18708,6 +18808,14 @@ public final class HBaseProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.srcChecksum_ = srcChecksum_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.versionMajor_ = versionMajor_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.versionMinor_ = versionMinor_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -18754,6 +18862,12 @@ public final class HBaseProtos {
           srcChecksum_ = other.srcChecksum_;
           onChanged();
         }
+        if (other.hasVersionMajor()) {
+          setVersionMajor(other.getVersionMajor());
+        }
+        if (other.hasVersionMinor()) {
+          setVersionMinor(other.getVersionMinor());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -19249,6 +19363,72 @@ public final class HBaseProtos {
         return this;
       }
 
+      // optional uint32 version_major = 7;
+      private int versionMajor_ ;
+      /**
+       * <code>optional uint32 version_major = 7;</code>
+       */
+      public boolean hasVersionMajor() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional uint32 version_major = 7;</code>
+       */
+      public int getVersionMajor() {
+        return versionMajor_;
+      }
+      /**
+       * <code>optional uint32 version_major = 7;</code>
+       */
+      public Builder setVersionMajor(int value) {
+        bitField0_ |= 0x00000040;
+        versionMajor_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 version_major = 7;</code>
+       */
+      public Builder clearVersionMajor() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        versionMajor_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional uint32 version_minor = 8;
+      private int versionMinor_ ;
+      /**
+       * <code>optional uint32 version_minor = 8;</code>
+       */
+      public boolean hasVersionMinor() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional uint32 version_minor = 8;</code>
+       */
+      public int getVersionMinor() {
+        return versionMinor_;
+      }
+      /**
+       * <code>optional uint32 version_minor = 8;</code>
+       */
+      public Builder setVersionMinor(int value) {
+        bitField0_ |= 0x00000080;
+        versionMinor_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 version_minor = 8;</code>
+       */
+      public Builder clearVersionMinor() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        versionMinor_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.VersionInfo)
     }
 
@@ -20106,19 +20286,20 @@ public final class HBaseProtos {
       "\022\026\n\016least_sig_bits\030\001 \002(\004\022\025\n\rmost_sig_bit" +
       "s\030\002 \002(\004\"T\n\023NamespaceDescriptor\022\014\n\004name\030\001" +
       " \002(\014\022/\n\rconfiguration\030\002 \003(\0132\030.hbase.pb.N",
-      "ameStringPair\"o\n\013VersionInfo\022\017\n\007version\030" +
-      "\001 \002(\t\022\013\n\003url\030\002 \002(\t\022\020\n\010revision\030\003 \002(\t\022\014\n\004" +
-      "user\030\004 \002(\t\022\014\n\004date\030\005 \002(\t\022\024\n\014src_checksum" +
-      "\030\006 \002(\t\"Q\n\020RegionServerInfo\022\020\n\010infoPort\030\001" +
-      " \001(\005\022+\n\014version_info\030\002 \001(\0132\025.hbase.pb.Ve" +
-      "rsionInfo*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLE" +
-      "SS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022" +
-      "\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO" +
-      "_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSECONDS\020\001\022\020\n\014MI" +
-      "CROSECONDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n\007SECOND",
-      "S\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DAYS\020\007B>\n" +
-      "*org.apache.hadoop.hbase.protobuf.genera" +
-      "tedB\013HBaseProtosH\001\240\001\001"
+      "ameStringPair\"\235\001\n\013VersionInfo\022\017\n\007version" +
+      "\030\001 \002(\t\022\013\n\003url\030\002 \002(\t\022\020\n\010revision\030\003 \002(\t\022\014\n" +
+      "\004user\030\004 \002(\t\022\014\n\004date\030\005 \002(\t\022\024\n\014src_checksu" +
+      "m\030\006 \002(\t\022\025\n\rversion_major\030\007 \001(\r\022\025\n\rversio" +
+      "n_minor\030\010 \001(\r\"Q\n\020RegionServerInfo\022\020\n\010inf" +
+      "oPort\030\001 \001(\005\022+\n\014version_info\030\002 \001(\0132\025.hbas" +
+      "e.pb.VersionInfo*r\n\013CompareType\022\010\n\004LESS\020" +
+      "\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_E" +
+      "QUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020" +
+      "\005\022\t\n\005NO_OP\020\006*n\n\010TimeUnit\022\017\n\013NANOSECONDS\020",
+      "\001\022\020\n\014MICROSECONDS\020\002\022\020\n\014MILLISECONDS\020\003\022\013\n" +
+      "\007SECONDS\020\004\022\013\n\007MINUTES\020\005\022\t\n\005HOURS\020\006\022\010\n\004DA" +
+      "YS\020\007B>\n*org.apache.hadoop.hbase.protobuf" +
+      ".generatedB\013HBaseProtosH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -20274,7 +20455,7 @@ public final class HBaseProtos {
           internal_static_hbase_pb_VersionInfo_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_VersionInfo_descriptor,
-              new java.lang.String[] { "Version", "Url", "Revision", "User", "Date", "SrcChecksum", });
+              new java.lang.String[] { "Version", "Url", "Revision", "User", "Date", "SrcChecksum", "VersionMajor", "VersionMinor", });
           internal_static_hbase_pb_RegionServerInfo_descriptor =
             getDescriptor().getMessageTypes().get(25);
           internal_static_hbase_pb_RegionServerInfo_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-protocol/src/main/protobuf/HBase.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/HBase.proto b/hbase-protocol/src/main/protobuf/HBase.proto
index 6cb42b1..e5c967a 100644
--- a/hbase-protocol/src/main/protobuf/HBase.proto
+++ b/hbase-protocol/src/main/protobuf/HBase.proto
@@ -245,6 +245,8 @@ message VersionInfo {
   required string user = 4;
   required string date = 5;
   required string src_checksum = 6;
+  optional uint32 version_major = 7;
+  optional uint32 version_minor = 8;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
index c405518..618777b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
@@ -44,6 +44,14 @@ public final class VersionInfoUtil {
                                           int major,
                                           int minor) {
     if (versionInfo != null) {
+      if (versionInfo.hasVersionMajor() && versionInfo.hasVersionMinor()) {
+        int clientMajor = versionInfo.getVersionMajor();
+        if (clientMajor != major) {
+          return clientMajor > major;
+        }
+        int clientMinor = versionInfo.getVersionMinor();
+        return clientMinor >= minor;
+      }
       try {
         String[] components = versionInfo.getVersion().split("\\.");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/408666a0/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index c82d49c..3b254c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
@@ -2100,7 +2101,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             ProtobufUtil.toResult(existence, region.getRegionInfo().getReplicaId() != 0);
         builder.setResult(pbr);
       } else if (r != null) {
-        ClientProtos.Result pbr = ProtobufUtil.toResult(r);
+        ClientProtos.Result pbr;
+        RpcCallContext call = RpcServer.getCurrentCall();
+        if (isClientCellBlockSupport(call) && controller instanceof PayloadCarryingRpcController
+            && VersionInfoUtil.hasMinimumVersion(call.getClientVersionInfo(), 1, 3)) {
+          pbr = ProtobufUtil.toResultNoData(r);
+          ((PayloadCarryingRpcController) controller).setCellScanner(CellUtil.createCellScanner(r
+              .rawCells()));
+        } else {
+          pbr = ProtobufUtil.toResult(r);
+        }
         builder.setResult(pbr);
       }
       if (r != null) {


[40/43] hbase git commit: HBASE-15026 The default value of "hbase.regions.slop" in docs is obsolete

Posted by sy...@apache.org.
HBASE-15026 The default value of "hbase.regions.slop" in docs is obsolete


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

Branch: refs/heads/hbase-12439
Commit: f53542c47d7f3d3de1195d4818545d79046e1d67
Parents: 1c7e9ec
Author: tedyu <yu...@gmail.com>
Authored: Fri Dec 25 15:37:31 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Dec 25 15:37:31 2015 -0800

----------------------------------------------------------------------
 hbase-common/src/main/resources/hbase-default.xml | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f53542c4/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 37a6298..d9a1994 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -586,8 +586,10 @@ possible configurations would overwhelm and obscure the important.
   </property>
   <property>
     <name>hbase.regions.slop</name>
-    <value>0.2</value>
-    <description>Rebalance if any regionserver has average + (average * slop) regions.</description>
+    <value>0.001</value>
+    <description>Rebalance if any regionserver has average + (average * slop) regions. 
+      The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load balancer),
+      while the default is 0.2 in other load balancers (i.e., SimpleLoadBalancer).</description>
   </property>
   <property>
     <name>hbase.server.thread.wakefrequency</name>


[15/43] hbase git commit: Revert "HBASE-14979 Update to the newest Zookeeper release"

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

This reverts commit ef92a6a067230cae96d3e3267da5a18ac5ad89c6.


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

Branch: refs/heads/hbase-12439
Commit: eb59d4d7690bb4d8d0afa460202c68885ef6a271
Parents: 48113d7
Author: stack <st...@apache.org>
Authored: Fri Dec 18 12:48:49 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Dec 18 12:48:49 2015 -0800

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


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


[17/43] hbase git commit: HBASE-14822 Renewing leases of scanners doesn't work.

Posted by sy...@apache.org.
HBASE-14822 Renewing leases of scanners doesn't work.


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

Branch: refs/heads/hbase-12439
Commit: 86a417eeade1fc1b0a7799a48694e6f1d9b4cb1c
Parents: cdca22a
Author: Lars Hofhansl <la...@apache.org>
Authored: Sat Dec 19 09:55:03 2015 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Sat Dec 19 09:55:03 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClientScanner.java      |   4 +-
 .../hadoop/hbase/client/ScannerCallable.java    |  12 +-
 .../client/ScannerCallableWithReplicas.java     |   4 +
 .../hadoop/hbase/protobuf/RequestConverter.java |   4 +-
 .../hbase/protobuf/generated/ClientProtos.java  | 218 +++++++++++++------
 hbase-protocol/src/main/protobuf/Client.proto   |   1 +
 .../hbase/regionserver/RSRpcServices.java       |   7 +
 .../hbase/client/TestFromClientSide3.java       |  25 ---
 .../hadoop/hbase/client/TestLeaseRenewal.java   | 125 +++++++++++
 9 files changed, 307 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index a4514bf..1658e5b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -776,13 +776,13 @@ public abstract class ClientScanner extends AbstractClientScanner {
   public boolean renewLease() {
     if (callable != null) {
       // do not return any rows, do not advance the scanner
-      callable.setCaching(0);
+      callable.setRenew(true);
       try {
         this.caller.callWithoutRetries(callable, this.scannerTimeout);
       } catch (Exception e) {
         return false;
       } finally {
-        callable.setCaching(this.caching);
+        callable.setRenew(false);
       }
       return true;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index 65f74c8..f6445a6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -71,6 +71,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   protected long scannerId = -1L;
   protected boolean instantiated = false;
   protected boolean closed = false;
+  protected boolean renew = false;
   private Scan scan;
   private int caching = 1;
   protected final ClusterConnection cConnection;
@@ -206,7 +207,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
           incRPCcallsMetrics();
           request =
               RequestConverter.buildScanRequest(scannerId, caching, false, nextCallSeq,
-                this.scanMetrics != null);
+                this.scanMetrics != null, renew);
           ScanResponse response = null;
           controller = controllerFactory.newController();
           controller.setPriority(getTableName());
@@ -411,6 +412,15 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
   }
 
   /**
+   * Indicate whether we make a call only to renew the lease, but without affected the scanner in
+   * any other way.
+   * @param val true if only the lease should be renewed
+   */
+  public void setRenew(boolean val) {
+    this.renew = val;
+  }
+
+  /**
    * @return the HRegionInfo for the current region
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 97d7d41..a197e90 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -98,6 +98,10 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
     currentScannerCallable.setClose();
   }
 
+  public void setRenew(boolean val) {
+    currentScannerCallable.setRenew(val);
+  }
+
   public void setCaching(int caching) {
     currentScannerCallable.setCaching(caching);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index ada510b..bd4c427 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -497,7 +497,8 @@ public final class RequestConverter {
    * @return a scan request
    */
   public static ScanRequest buildScanRequest(final long scannerId, final int numberOfRows,
-      final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics) {
+      final boolean closeScanner, final long nextCallSeq, final boolean trackMetrics,
+      final boolean renew) {
     ScanRequest.Builder builder = ScanRequest.newBuilder();
     builder.setNumberOfRows(numberOfRows);
     builder.setCloseScanner(closeScanner);
@@ -506,6 +507,7 @@ public final class RequestConverter {
     builder.setClientHandlesPartials(true);
     builder.setClientHandlesHeartbeats(true);
     builder.setTrackScanMetrics(trackMetrics);
+    builder.setRenew(renew);
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
index 5e17ad5..d188531 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
@@ -17171,6 +17171,16 @@ public final class ClientProtos {
      * <code>optional bool track_scan_metrics = 9;</code>
      */
     boolean getTrackScanMetrics();
+
+    // optional bool renew = 10 [default = false];
+    /**
+     * <code>optional bool renew = 10 [default = false];</code>
+     */
+    boolean hasRenew();
+    /**
+     * <code>optional bool renew = 10 [default = false];</code>
+     */
+    boolean getRenew();
   }
   /**
    * Protobuf type {@code hbase.pb.ScanRequest}
@@ -17297,6 +17307,11 @@ public final class ClientProtos {
               trackScanMetrics_ = input.readBool();
               break;
             }
+            case 80: {
+              bitField0_ |= 0x00000200;
+              renew_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -17493,6 +17508,22 @@ public final class ClientProtos {
       return trackScanMetrics_;
     }
 
+    // optional bool renew = 10 [default = false];
+    public static final int RENEW_FIELD_NUMBER = 10;
+    private boolean renew_;
+    /**
+     * <code>optional bool renew = 10 [default = false];</code>
+     */
+    public boolean hasRenew() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>optional bool renew = 10 [default = false];</code>
+     */
+    public boolean getRenew() {
+      return renew_;
+    }
+
     private void initFields() {
       region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
       scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance();
@@ -17503,6 +17534,7 @@ public final class ClientProtos {
       clientHandlesPartials_ = false;
       clientHandlesHeartbeats_ = false;
       trackScanMetrics_ = false;
+      renew_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -17555,6 +17587,9 @@ public final class ClientProtos {
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
         output.writeBool(9, trackScanMetrics_);
       }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeBool(10, renew_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -17600,6 +17635,10 @@ public final class ClientProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(9, trackScanMetrics_);
       }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(10, renew_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -17668,6 +17707,11 @@ public final class ClientProtos {
         result = result && (getTrackScanMetrics()
             == other.getTrackScanMetrics());
       }
+      result = result && (hasRenew() == other.hasRenew());
+      if (hasRenew()) {
+        result = result && (getRenew()
+            == other.getRenew());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -17717,6 +17761,10 @@ public final class ClientProtos {
         hash = (37 * hash) + TRACK_SCAN_METRICS_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getTrackScanMetrics());
       }
+      if (hasRenew()) {
+        hash = (37 * hash) + RENEW_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getRenew());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -17867,6 +17915,8 @@ public final class ClientProtos {
         bitField0_ = (bitField0_ & ~0x00000080);
         trackScanMetrics_ = false;
         bitField0_ = (bitField0_ & ~0x00000100);
+        renew_ = false;
+        bitField0_ = (bitField0_ & ~0x00000200);
         return this;
       }
 
@@ -17939,6 +17989,10 @@ public final class ClientProtos {
           to_bitField0_ |= 0x00000100;
         }
         result.trackScanMetrics_ = trackScanMetrics_;
+        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        result.renew_ = renew_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -17982,6 +18036,9 @@ public final class ClientProtos {
         if (other.hasTrackScanMetrics()) {
           setTrackScanMetrics(other.getTrackScanMetrics());
         }
+        if (other.hasRenew()) {
+          setRenew(other.getRenew());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -18486,6 +18543,39 @@ public final class ClientProtos {
         return this;
       }
 
+      // optional bool renew = 10 [default = false];
+      private boolean renew_ ;
+      /**
+       * <code>optional bool renew = 10 [default = false];</code>
+       */
+      public boolean hasRenew() {
+        return ((bitField0_ & 0x00000200) == 0x00000200);
+      }
+      /**
+       * <code>optional bool renew = 10 [default = false];</code>
+       */
+      public boolean getRenew() {
+        return renew_;
+      }
+      /**
+       * <code>optional bool renew = 10 [default = false];</code>
+       */
+      public Builder setRenew(boolean value) {
+        bitField0_ |= 0x00000200;
+        renew_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool renew = 10 [default = false];</code>
+       */
+      public Builder clearRenew() {
+        bitField0_ = (bitField0_ & ~0x00000200);
+        renew_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest)
     }
 
@@ -34156,76 +34246,76 @@ public final class ClientProtos {
       " \001(\0162\025.hbase.pb.Consistency:\006STRONG\022\017\n\007c" +
       "aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
       "\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
-      "lumnFamilyTimeRange\"\220\002\n\013ScanRequest\022)\n\006r" +
+      "lumnFamilyTimeRange\"\246\002\n\013ScanRequest\022)\n\006r" +
       "egion\030\001 \001(\0132\031.hbase.pb.RegionSpecifier\022\034",
       "\n\004scan\030\002 \001(\0132\016.hbase.pb.Scan\022\022\n\nscanner_" +
       "id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclos" +
       "e_scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037" +
       "\n\027client_handles_partials\030\007 \001(\010\022!\n\031clien" +
       "t_handles_heartbeats\030\010 \001(\010\022\032\n\022track_scan" +
-      "_metrics\030\t \001(\010\"\232\002\n\014ScanResponse\022\030\n\020cells" +
-      "_per_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n" +
-      "\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007resu" +
-      "lts\030\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001" +
-      "(\010\022\037\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026m",
-      "ore_results_in_region\030\010 \001(\010\022\031\n\021heartbeat" +
-      "_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.h" +
-      "base.pb.ScanMetrics\"\305\001\n\024BulkLoadHFileReq" +
-      "uest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSp" +
-      "ecifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb." +
-      "BulkLoadHFileRequest.FamilyPath\022\026\n\016assig" +
-      "n_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030" +
-      "\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResp" +
-      "onse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorServi" +
-      "ceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(",
-      "\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"" +
-      "B\n\030CoprocessorServiceResult\022&\n\005value\030\001 \001" +
-      "(\0132\027.hbase.pb.NameBytesPair\"v\n\031Coprocess" +
-      "orServiceRequest\022)\n\006region\030\001 \002(\0132\031.hbase" +
-      ".pb.RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbas" +
-      "e.pb.CoprocessorServiceCall\"o\n\032Coprocess" +
-      "orServiceResponse\022)\n\006region\030\001 \002(\0132\031.hbas" +
-      "e.pb.RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hb" +
-      "ase.pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index" +
-      "\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Muta",
-      "tionProto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n" +
-      "\014service_call\030\004 \001(\0132 .hbase.pb.Coprocess" +
-      "orServiceCall\"k\n\014RegionAction\022)\n\006region\030" +
-      "\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atom" +
-      "ic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Acti" +
-      "on\"c\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 " +
-      "\001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022comp" +
-      "actionPressure\030\003 \001(\005:\0010\"\332\001\n\021ResultOrExce" +
-      "ption\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.h" +
-      "base.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbas",
-      "e.pb.NameBytesPair\022:\n\016service_result\030\004 \001" +
-      "(\0132\".hbase.pb.CoprocessorServiceResult\022," +
-      "\n\tloadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadS" +
-      "tats\"x\n\022RegionActionResult\0226\n\021resultOrEx" +
-      "ception\030\001 \003(\0132\033.hbase.pb.ResultOrExcepti" +
-      "on\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameByt" +
-      "esPair\"x\n\014MultiRequest\022,\n\014regionAction\030\001" +
-      " \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceGro" +
-      "up\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb.C" +
-      "ondition\"\\\n\rMultiResponse\0228\n\022regionActio",
-      "nResult\030\001 \003(\0132\034.hbase.pb.RegionActionRes" +
-      "ult\022\021\n\tprocessed\030\002 \001(\010*\'\n\013Consistency\022\n\n" +
-      "\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientServic" +
-      "e\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hbase.p" +
-      "b.GetResponse\022;\n\006Mutate\022\027.hbase.pb.Mutat" +
-      "eRequest\032\030.hbase.pb.MutateResponse\0225\n\004Sc" +
-      "an\022\025.hbase.pb.ScanRequest\032\026.hbase.pb.Sca" +
-      "nResponse\022P\n\rBulkLoadHFile\022\036.hbase.pb.Bu" +
-      "lkLoadHFileRequest\032\037.hbase.pb.BulkLoadHF" +
-      "ileResponse\022X\n\013ExecService\022#.hbase.pb.Co",
-      "processorServiceRequest\032$.hbase.pb.Copro" +
-      "cessorServiceResponse\022d\n\027ExecRegionServe" +
-      "rService\022#.hbase.pb.CoprocessorServiceRe" +
+      "_metrics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\"\232\002\n" +
+      "\014ScanResponse\022\030\n\020cells_per_result\030\001 \003(\r\022" +
+      "\022\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(" +
+      "\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase." +
+      "pb.Result\022\r\n\005stale\030\006 \001(\010\022\037\n\027partial_flag",
+      "_per_result\030\007 \003(\010\022\036\n\026more_results_in_reg" +
+      "ion\030\010 \001(\010\022\031\n\021heartbeat_message\030\t \001(\010\022+\n\014" +
+      "scan_metrics\030\n \001(\0132\025.hbase.pb.ScanMetric" +
+      "s\"\305\001\n\024BulkLoadHFileRequest\022)\n\006region\030\001 \002" +
+      "(\0132\031.hbase.pb.RegionSpecifier\022>\n\013family_" +
+      "path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReque" +
+      "st.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n" +
+      "\nFamilyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(" +
+      "\t\"\'\n\025BulkLoadHFileResponse\022\016\n\006loaded\030\001 \002" +
+      "(\010\"a\n\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(",
+      "\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013method_name\030\003" +
+      " \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorServ" +
+      "iceResult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.Name" +
+      "BytesPair\"v\n\031CoprocessorServiceRequest\022)" +
+      "\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifie" +
+      "r\022.\n\004call\030\002 \002(\0132 .hbase.pb.CoprocessorSe" +
+      "rviceCall\"o\n\032CoprocessorServiceResponse\022" +
+      ")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
+      "er\022&\n\005value\030\002 \002(\0132\027.hbase.pb.NameBytesPa" +
+      "ir\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutation",
+      "\030\002 \001(\0132\027.hbase.pb.MutationProto\022\032\n\003get\030\003" +
+      " \001(\0132\r.hbase.pb.Get\0226\n\014service_call\030\004 \001(" +
+      "\0132 .hbase.pb.CoprocessorServiceCall\"k\n\014R" +
+      "egionAction\022)\n\006region\030\001 \002(\0132\031.hbase.pb.R" +
+      "egionSpecifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006action" +
+      "\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017RegionLoadSt" +
+      "ats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccu" +
+      "pancy\030\002 \001(\005:\0010\022\035\n\022compactionPressure\030\003 \001" +
+      "(\005:\0010\"\332\001\n\021ResultOrException\022\r\n\005index\030\001 \001" +
+      "(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*\n\t",
+      "exception\030\003 \001(\0132\027.hbase.pb.NameBytesPair" +
+      "\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Copr" +
+      "ocessorServiceResult\022,\n\tloadStats\030\005 \001(\0132" +
+      "\031.hbase.pb.RegionLoadStats\"x\n\022RegionActi" +
+      "onResult\0226\n\021resultOrException\030\001 \003(\0132\033.hb" +
+      "ase.pb.ResultOrException\022*\n\texception\030\002 " +
+      "\001(\0132\027.hbase.pb.NameBytesPair\"x\n\014MultiReq" +
+      "uest\022,\n\014regionAction\030\001 \003(\0132\026.hbase.pb.Re" +
+      "gionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcondit" +
+      "ion\030\003 \001(\0132\023.hbase.pb.Condition\"\\\n\rMultiR",
+      "esponse\0228\n\022regionActionResult\030\001 \003(\0132\034.hb" +
+      "ase.pb.RegionActionResult\022\021\n\tprocessed\030\002" +
+      " \001(\010*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMEL" +
+      "INE\020\0012\203\004\n\rClientService\0222\n\003Get\022\024.hbase.p" +
+      "b.GetRequest\032\025.hbase.pb.GetResponse\022;\n\006M" +
+      "utate\022\027.hbase.pb.MutateRequest\032\030.hbase.p" +
+      "b.MutateResponse\0225\n\004Scan\022\025.hbase.pb.Scan" +
+      "Request\032\026.hbase.pb.ScanResponse\022P\n\rBulkL" +
+      "oadHFile\022\036.hbase.pb.BulkLoadHFileRequest" +
+      "\032\037.hbase.pb.BulkLoadHFileResponse\022X\n\013Exe",
+      "cService\022#.hbase.pb.CoprocessorServiceRe" +
       "quest\032$.hbase.pb.CoprocessorServiceRespo" +
-      "nse\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.h" +
-      "base.pb.MultiResponseBB\n*org.apache.hado" +
-      "op.hbase.protobuf.generatedB\014ClientProto" +
-      "sH\001\210\001\001\240\001\001"
+      "nse\022d\n\027ExecRegionServerService\022#.hbase.p" +
+      "b.CoprocessorServiceRequest\032$.hbase.pb.C" +
+      "oprocessorServiceResponse\0228\n\005Multi\022\026.hba" +
+      "se.pb.MultiRequest\032\027.hbase.pb.MultiRespo" +
+      "nseBB\n*org.apache.hadoop.hbase.protobuf." +
+      "generatedB\014ClientProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -34321,7 +34411,7 @@ public final class ClientProtos {
           internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ScanRequest_descriptor,
-              new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", });
+              new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", });
           internal_static_hbase_pb_ScanResponse_descriptor =
             getDescriptor().getMessageTypes().get(13);
           internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-protocol/src/main/protobuf/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto
index a3a969f..ca9abdc 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -277,6 +277,7 @@ message ScanRequest {
   optional bool client_handles_partials = 7;
   optional bool client_handles_heartbeats = 8;
   optional bool track_scan_metrics = 9;
+  optional bool renew = 10 [default = false];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 3b254c0..75705e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -2517,6 +2517,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         ttl = this.scannerLeaseTimeoutPeriod;
       }
       assert scanner != null;
+      if (request.hasRenew() && request.getRenew()) {
+        lease = regionServer.leases.removeLease(scannerName);
+        if (lease != null && scanners.containsKey(scannerName)) {
+          regionServer.leases.addLease(lease);
+        }
+        return builder.build();
+      }
       RpcCallContext context = RpcServer.getCurrentCall();
       Object lastBlock = null;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 96bc59f..c451cf0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -476,29 +476,4 @@ public class TestFromClientSide3 {
     assertTrue(Arrays.equals(res.getValue(FAMILY, COL_QUAL), VAL_BYTES));
     table.close();
   }
-
-  @Test
-  public void testLeaseRenewal() throws Exception {
-    Table table = TEST_UTIL.createTable(
-      TableName.valueOf("testLeaseRenewal"), FAMILY);
-    Put p = new Put(ROW_BYTES);
-    p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
-    table.put(p);
-    p = new Put(ANOTHERROW);
-    p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
-    table.put(p);
-    Scan s = new Scan();
-    s.setCaching(1);
-    ResultScanner rs = table.getScanner(s);
-    // make sure that calling renewLease does not impact the scan results
-    assertTrue(rs.renewLease());
-    assertTrue(Arrays.equals(rs.next().getRow(), ANOTHERROW));
-    assertTrue(rs.renewLease());
-    assertTrue(Arrays.equals(rs.next().getRow(), ROW_BYTES));
-    assertTrue(rs.renewLease());
-    assertNull(rs.next());
-    assertFalse(rs.renewLease());
-    rs.close();
-    table.close();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/86a417ee/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
new file mode 100644
index 0000000..c89edf5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -0,0 +1,125 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestLeaseRenewal {
+  final Log LOG = LogFactory.getLog(getClass());
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte[] ANOTHERROW = Bytes.toBytes("anotherrow");
+  private final static byte[] COL_QUAL = Bytes.toBytes("f1");
+  private final static byte[] VAL_BYTES = Bytes.toBytes("v1");
+  private final static byte[] ROW_BYTES = Bytes.toBytes("r1");
+  private final static int leaseTimeout =
+      HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD / 4;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
+      leaseTimeout);
+    TEST_UTIL.startMiniCluster();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before
+  public void setUp() throws Exception {
+    // Nothing to do.
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @After
+  public void tearDown() throws Exception {
+    for (HTableDescriptor htd : TEST_UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      TEST_UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  @Test
+  public void testLeaseRenewal() throws Exception {
+    HTable table = TEST_UTIL.createTable(
+      TableName.valueOf("testLeaseRenewal"), FAMILY);
+    Put p = new Put(ROW_BYTES);
+    p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
+    table.put(p);
+    p = new Put(ANOTHERROW);
+    p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
+    table.put(p);
+    Scan s = new Scan();
+    s.setCaching(1);
+    ResultScanner rs = table.getScanner(s);
+    // make sure that calling renewLease does not impact the scan results
+    assertTrue(rs.renewLease());
+    assertTrue(Arrays.equals(rs.next().getRow(), ANOTHERROW));
+    // renew the lease a few times, long enough to be sure
+    // the lease would have expired otherwise
+    Thread.sleep(leaseTimeout/2);
+    assertTrue(rs.renewLease());
+    Thread.sleep(leaseTimeout/2);
+    assertTrue(rs.renewLease());
+    Thread.sleep(leaseTimeout/2);
+    assertTrue(rs.renewLease());
+    // make sure we haven't advanced the scanner
+    assertTrue(Arrays.equals(rs.next().getRow(), ROW_BYTES));
+    assertTrue(rs.renewLease());
+    // make sure scanner is exhausted now
+    assertNull(rs.next());
+    // renewLease should return false now
+    assertFalse(rs.renewLease());
+    rs.close();
+    table.close();
+  }
+}


[26/43] hbase git commit: HBASE-15014 Fix filterCellByStore in WALsplitter is awful for performance

Posted by sy...@apache.org.
HBASE-15014 Fix filterCellByStore in WALsplitter is awful for performance


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

Branch: refs/heads/hbase-12439
Commit: 6e2c5d216eb1f4cacad7c5d7ed43b67785cabb67
Parents: 53e5d27
Author: Elliott Clark <ec...@apache.org>
Authored: Fri Dec 18 14:14:25 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Mon Dec 21 22:50:38 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/wal/WALEdit.java  | 14 +++++++++++++-
 .../apache/hadoop/hbase/wal/WALSplitter.java    | 20 +++++++++++++-------
 2 files changed, 26 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e2c5d21/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index 3b774ef..cea2ee7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -99,7 +99,7 @@ public class WALEdit implements Writable, HeapSize {
   private final int VERSION_2 = -1;
   private final boolean isReplay;
 
-  private final ArrayList<Cell> cells = new ArrayList<Cell>(1);
+  private ArrayList<Cell> cells = new ArrayList<Cell>(1);
 
   public static final WALEdit EMPTY_WALEDIT = new WALEdit();
 
@@ -170,6 +170,18 @@ public class WALEdit implements Writable, HeapSize {
     return cells;
   }
 
+  /**
+   * This is not thread safe.
+   * This will change the WALEdit and shouldn't be used unless you are sure that nothing
+   * else depends on the contents being immutable.
+   *
+   * @param cells the list of cells that this WALEdit now contains.
+   */
+  @InterfaceAudience.Private
+  public void setCells(ArrayList<Cell> cells) {
+    this.cells = cells;
+  }
+
   public NavigableMap<byte[], Integer> getAndRemoveScopes() {
     NavigableMap<byte[], Integer> result = scopes;
     scopes = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e2c5d21/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 04438fd..8abd950 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -1505,21 +1505,27 @@ public class WALSplitter {
       if (maxSeqIdInStores == null || maxSeqIdInStores.isEmpty()) {
         return;
       }
-      List<Cell> skippedCells = new ArrayList<Cell>();
+      // Create the array list for the cells that aren't filtered.
+      // We make the assumption that most cells will be kept.
+      ArrayList<Cell> keptCells = new ArrayList<Cell>(logEntry.getEdit().getCells().size());
       for (Cell cell : logEntry.getEdit().getCells()) {
-        if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+          keptCells.add(cell);
+        } else {
           byte[] family = CellUtil.cloneFamily(cell);
           Long maxSeqId = maxSeqIdInStores.get(family);
           // Do not skip cell even if maxSeqId is null. Maybe we are in a rolling upgrade,
           // or the master was crashed before and we can not get the information.
-          if (maxSeqId != null && maxSeqId.longValue() >= logEntry.getKey().getLogSeqNum()) {
-            skippedCells.add(cell);
+          if (maxSeqId == null || maxSeqId.longValue() < logEntry.getKey().getLogSeqNum()) {
+            keptCells.add(cell);
           }
         }
       }
-      if (!skippedCells.isEmpty()) {
-        logEntry.getEdit().getCells().removeAll(skippedCells);
-      }
+
+      // Anything in the keptCells array list is still live.
+      // So rather than removing the cells from the array list
+      // which would be an O(n^2) operation, we just replace the list
+      logEntry.getEdit().setCells(keptCells);
     }
 
     @Override


[38/43] hbase git commit: HBASE-15035 bulkloading hfiles with tags that require splits do not preserve tags

Posted by sy...@apache.org.
HBASE-15035 bulkloading hfiles with tags that require splits do not preserve tags


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

Branch: refs/heads/hbase-12439
Commit: 3de0b0417c032693e34eb7f61633fdbf042dc5b5
Parents: dfada43
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Fri Dec 25 09:51:34 2015 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Fri Dec 25 09:51:34 2015 -0800

----------------------------------------------------------------------
 .../hbase/io/hfile/HFileContextBuilder.java     |  2 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |  1 +
 .../mapreduce/TestLoadIncrementalHFiles.java    | 68 +++++++++++++-
 ...oadIncrementalHFilesUseSecurityEndPoint.java |  6 ++
 .../TestSecureLoadIncrementalHFiles.java        |  5 ++
 .../apache/hadoop/hbase/util/HFileTestUtil.java | 93 ++++++++++++++++++--
 6 files changed, 164 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
index ce3541f..6d3bb13 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
@@ -37,7 +37,7 @@ public class HFileContextBuilder {
   /** Whether mvcc is to be included in the Read/Write **/
   private boolean includesMvcc = true;
   /** Whether tags are to be included in the Read/Write **/
-  private boolean includesTags;
+  private boolean includesTags = false;
   /** Compression algorithm used **/
   private Algorithm compression = Algorithm.NONE;
   /** Whether tags to be compressed or not **/

http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 0e35427..172dfe3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -903,6 +903,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
                                   .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
                                   .withBlockSize(blocksize)
                                   .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding())
+                                  .withIncludesTags(true)
                                   .build();
       halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
           fs)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/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 fcf9ef2..8c64ff8 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
@@ -33,11 +33,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -50,8 +52,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
 /**
  * Test cases for the "load" half of the HFileOutputFormat bulk load
@@ -60,6 +64,9 @@ import org.junit.experimental.categories.Category;
  */
 @Category({MapReduceTests.class, LargeTests.class})
 public class TestLoadIncrementalHFiles {
+  @Rule
+  public TestName tn = new TestName();
+
   private static final byte[] QUALIFIER = Bytes.toBytes("myqual");
   private static final byte[] FAMILY = Bytes.toBytes("myfam");
   private static final String NAMESPACE = "bulkNS";
@@ -80,6 +87,9 @@ public class TestLoadIncrementalHFiles {
     util.getConfiguration().setInt(
       LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
       MAX_FILES_PER_REGION_PER_FAMILY);
+    // change default behavior so that tag values are returned with normal rpcs
+    util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
+        KeyValueCodecWithTags.class.getCanonicalName());
     util.startMiniCluster();
 
     setupNamespace();
@@ -224,6 +234,14 @@ public class TestLoadIncrementalHFiles {
     );
   }
 
+  private HTableDescriptor buildHTD(TableName tableName, BloomType bloomType) {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
+    familyDesc.setBloomFilterType(bloomType);
+    htd.addFamily(familyDesc);
+    return htd;
+  }
+
   private void runTest(String testName, BloomType bloomType,
       byte[][][] hfileRanges) throws Exception {
     runTest(testName, bloomType, null, hfileRanges);
@@ -245,10 +263,7 @@ public class TestLoadIncrementalHFiles {
 
   private void runTest(String testName, TableName tableName, BloomType bloomType,
       boolean preCreateTable, byte[][] tableSplitKeys, byte[][][] hfileRanges) throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
-    familyDesc.setBloomFilterType(bloomType);
-    htd.addFamily(familyDesc);
+    HTableDescriptor htd = buildHTD(tableName, bloomType);
     runTest(testName, htd, bloomType, preCreateTable, tableSplitKeys, hfileRanges);
   }
 
@@ -298,6 +313,51 @@ public class TestLoadIncrementalHFiles {
   }
 
   /**
+   * Test that tags survive through a bulk load that needs to split hfiles.
+   *
+   * This test depends on the "hbase.client.rpc.codec" =  KeyValueCodecWithTags so that the client
+   * can get tags in the responses.
+   */
+  @Test(timeout = 60000)
+  public void testTagsSurviveBulkLoadSplit() throws Exception {
+    Path dir = util.getDataTestDirOnTestFS(tn.getMethodName());
+    FileSystem fs = util.getTestFileSystem();
+    dir = dir.makeQualified(fs);
+    Path familyDir = new Path(dir, Bytes.toString(FAMILY));
+    // table has these split points
+    byte [][] tableSplitKeys = new byte[][] {
+            Bytes.toBytes("aaa"), Bytes.toBytes("fff"), Bytes.toBytes("jjj"),
+            Bytes.toBytes("ppp"), Bytes.toBytes("uuu"), Bytes.toBytes("zzz"),
+    };
+
+    // creating an hfile that has values that span the split points.
+    byte[] from = Bytes.toBytes("ddd");
+    byte[] to = Bytes.toBytes("ooo");
+    HFileTestUtil.createHFileWithTags(util.getConfiguration(), fs,
+        new Path(familyDir, tn.getMethodName()+"_hfile"),
+        FAMILY, QUALIFIER, from, to, 1000);
+    int expectedRows = 1000;
+
+    TableName tableName = TableName.valueOf(tn.getMethodName());
+    HTableDescriptor htd = buildHTD(tableName, BloomType.NONE);
+    util.getAdmin().createTable(htd, tableSplitKeys);
+
+    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
+    String [] args= {dir.toString(), tableName.toString()};
+    loader.run(args);
+
+    Table table = util.getConnection().getTable(tableName);
+    try {
+      assertEquals(expectedRows, util.countRows(table));
+      HFileTestUtil.verifyTags(table);
+    } finally {
+      table.close();
+    }
+
+    util.deleteTable(tableName);
+  }
+
+  /**
    * Test loading into a column family that does not exist.
    */
   @Test(timeout = 60000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesUseSecurityEndPoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesUseSecurityEndPoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesUseSecurityEndPoint.java
index 6a916f6..11627a1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesUseSecurityEndPoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesUseSecurityEndPoint.java
@@ -19,6 +19,8 @@
 
 package org.apache.hadoop.hbase.mapreduce;
 
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.junit.BeforeClass;
@@ -33,6 +35,10 @@ public class TestLoadIncrementalHFilesUseSecurityEndPoint extends TestLoadIncrem
       MAX_FILES_PER_REGION_PER_FAMILY);
     util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
       "org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
+    // change default behavior so that tag values are returned with normal rpcs
+    util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
+        KeyValueCodecWithTags.class.getCanonicalName());
+
     util.startMiniCluster();
     setupNamespace();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java
index e8aca29..78fddbc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java
@@ -19,6 +19,8 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -52,6 +54,9 @@ public class TestSecureLoadIncrementalHFiles extends  TestLoadIncrementalHFiles{
     util.getConfiguration().setInt(
         LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY,
         MAX_FILES_PER_REGION_PER_FAMILY);
+    // change default behavior so that tag values are returned with normal rpcs
+    util.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
+        KeyValueCodecWithTags.class.getCanonicalName());
 
     util.startMiniCluster();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3de0b041/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
index f52837b..fdf4fd9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
@@ -21,15 +21,26 @@ package org.apache.hadoop.hbase.util;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
 /**
  * Utility class for HFile-related testing.
  */
@@ -37,15 +48,45 @@ public class HFileTestUtil {
 
   /**
    * Create an HFile with the given number of rows between a given
-   * start key and end key.
+   * start key and end key @ family:qualifier.  The value will be the key value.
+   * This file will not have tags.
    */
   public static void createHFile(
       Configuration configuration,
       FileSystem fs, Path path,
       byte[] family, byte[] qualifier,
-      byte[] startKey, byte[] endKey, int numRows) throws IOException
-  {
-    HFileContext meta = new HFileContextBuilder().build();
+      byte[] startKey, byte[] endKey, int numRows) throws IOException {
+    createHFile(configuration, fs, path, family, qualifier, startKey, endKey,
+        numRows, false);
+  }
+
+  /**
+   * Create an HFile with the given number of rows between a given
+   * start key and end key @ family:qualifier.  The value will be the key value.
+   * This cells will also have a tag whose value is the key.
+   */
+  public static void createHFileWithTags(
+      Configuration configuration,
+      FileSystem fs, Path path,
+      byte[] family, byte[] qualifier,
+      byte[] startKey, byte[] endKey, int numRows) throws IOException {
+    createHFile(configuration, fs, path, family, qualifier, startKey, endKey, numRows, true);
+  }
+
+  /**
+   * Create an HFile with the given number of rows between a given
+   * start key and end key @ family:qualifier.
+   * If withTag is true, we add the rowKey as the tag value for
+   * tagtype MOB_TABLE_NAME_TAG_TYPE
+   */
+  public static void createHFile(
+      Configuration configuration,
+      FileSystem fs, Path path,
+      byte[] family, byte[] qualifier,
+      byte[] startKey, byte[] endKey, int numRows, boolean withTag) throws IOException {
+    HFileContext meta = new HFileContextBuilder()
+        .withIncludesTags(withTag)
+        .build();
     HFile.Writer writer = HFile.getWriterFactory(configuration, new CacheConfig(configuration))
         .withPath(fs, path)
         .withFileContext(meta)
@@ -53,8 +94,22 @@ public class HFileTestUtil {
     long now = System.currentTimeMillis();
     try {
       // subtract 2 since iterateOnSplits doesn't include boundary keys
-      for (byte[] key : Bytes.iterateOnSplits(startKey, endKey, numRows-2)) {
+      for (byte[] key : Bytes.iterateOnSplits(startKey, endKey, numRows - 2)) {
         KeyValue kv = new KeyValue(key, family, qualifier, now, key);
+        if (withTag) {
+          // add a tag.  Arbitrarily chose mob tag since we have a helper already.
+          Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, key);
+          kv = MobUtils.createMobRefKeyValue(kv, key, tableNameTag);
+
+          // verify that the kv has the tag.
+          byte[] ta = kv.getTagsArray();
+          int toff = kv.getTagsOffset();
+          int tlen = kv.getTagsLength();
+          Tag t = Tag.getTag(ta, toff, tlen, TagType.MOB_TABLE_NAME_TAG_TYPE);
+          if (t == null) {
+            throw new IllegalStateException("Tag didn't stick to KV " + kv.toString());
+          }
+        }
         writer.append(kv);
       }
     } finally {
@@ -63,4 +118,30 @@ public class HFileTestUtil {
       writer.close();
     }
   }
-}
+
+  /**
+   * This verifies that each cell has a tag that is equal to its rowkey name.  For this to work
+   * the hbase instance must have HConstants.RPC_CODEC_CONF_KEY set to
+   * KeyValueCodecWithTags.class.getCanonicalName());
+   * @param table table containing tagged cells
+   * @throws IOException if problems reading table
+   */
+  public static void verifyTags(Table table) throws IOException {
+    ResultScanner s = table.getScanner(new Scan());
+    for (Result r : s) {
+      for (Cell c : r.listCells()) {
+        byte[] ta = c.getTagsArray();
+        int toff = c.getTagsOffset();
+        int tlen = c.getTagsLength();
+        Tag t = Tag.getTag(ta, toff, tlen, TagType.MOB_TABLE_NAME_TAG_TYPE);
+        if (t == null) {
+          fail(c.toString() + " has null tag");
+          continue;
+        }
+        byte[] tval = t.getValue();
+        assertArrayEquals(c.toString() + " has tag" + Bytes.toString(tval),
+            r.getRow(), tval);
+      }
+    }
+  }
+}
\ No newline at end of file


[18/43] hbase git commit: HBASE-15015 Checktyle plugin shouldn't check Jamon-generated Java classes

Posted by sy...@apache.org.
HBASE-15015 Checktyle plugin shouldn't check Jamon-generated Java classes


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

Branch: refs/heads/hbase-12439
Commit: b68542e35a206a93262fab03fd1c2d8f481cf05e
Parents: 86a417e
Author: Mikhail Antonov <an...@apache.org>
Authored: Sat Dec 19 11:58:15 2015 -0800
Committer: Mikhail Antonov <an...@apache.org>
Committed: Sat Dec 19 11:58:15 2015 -0800

----------------------------------------------------------------------
 .../src/main/resources/hbase/checkstyle-suppressions.xml            | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b68542e3/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
index 5f3f586..46009e9 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
@@ -32,5 +32,6 @@
 -->
 <suppressions>
   <suppress checks="." files=".*/generated/.*\.java"/>
+  <suppress checks="." files=".*/generated-jamon/.*\.java"/>
   <suppress checks="MagicNumberCheck" files=".*Test\.java"/>
 </suppressions>


[06/43] hbase git commit: HBASE-14978 Don't allow Multi to retain too many blocks

Posted by sy...@apache.org.
HBASE-14978 Don't allow Multi to retain too many blocks


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

Branch: refs/heads/hbase-12439
Commit: 217036d81693f3806d271f6f5d5d7d8f17823745
Parents: bbfff0d
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Dec 14 18:16:03 2015 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Thu Dec 17 16:13:50 2015 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |  3 +
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 11 +++
 .../hbase/regionserver/RSRpcServices.java       | 71 +++++++++++++-----
 .../hbase/client/TestMultiRespectsLimits.java   | 75 ++++++++++++++++++--
 4 files changed, 137 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/217036d8/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index d14e9b2..f41dfbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -90,4 +90,7 @@ public interface RpcCallContext extends Delayable {
    * onerous.
    */
   void incrementResponseCellSize(long cellSize);
+
+  long getResponseBlockSize();
+  void incrementResponseBlockSize(long blockSize);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/217036d8/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 2bef247..ed8d37d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -319,6 +319,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     private RpcCallback callback;
 
     private long responseCellSize = 0;
+    private long responseBlockSize = 0;
     private boolean retryImmediatelySupported;
 
     Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
@@ -551,6 +552,16 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       responseCellSize += cellSize;
     }
 
+    @Override
+    public long getResponseBlockSize() {
+      return responseBlockSize;
+    }
+
+    @Override
+    public void incrementResponseBlockSize(long blockSize) {
+      responseBlockSize += blockSize;
+    }
+
     /**
      * If we have a response, and delay is not set, then respond
      * immediately.  Otherwise, do not respond to client.  This is

http://git-wip-us.apache.org/repos/asf/hbase/blob/217036d8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index bba38f79..c82d49c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -23,6 +23,7 @@ import java.io.InterruptedIOException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -39,6 +40,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ByteBufferedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
@@ -658,6 +660,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     List<ClientProtos.Action> mutations = null;
     long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
     IOException sizeIOE = null;
+    Object lastBlock = null;
     for (ClientProtos.Action action : actions.getActionList()) {
       ClientProtos.ResultOrException.Builder resultOrExceptionBuilder = null;
       try {
@@ -665,7 +668,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
 
         if (context != null
             && context.isRetryImmediatelySupported()
-            && context.getResponseCellSize() > maxQuotaResultSize) {
+            && (context.getResponseCellSize() > maxQuotaResultSize
+              || context.getResponseBlockSize() > maxQuotaResultSize)) {
 
           // We're storing the exception since the exception and reason string won't
           // change after the response size limit is reached.
@@ -674,15 +678,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             // Throwing will kill the JVM's JIT.
             //
             // Instead just create the exception and then store it.
-            sizeIOE = new MultiActionResultTooLarge("Max response size exceeded: "
-                    + context.getResponseCellSize());
+            sizeIOE = new MultiActionResultTooLarge("Max size exceeded"
+                + " CellSize: " + context.getResponseCellSize()
+                + " BlockSize: " + context.getResponseBlockSize());
 
             // Only report the exception once since there's only one request that
             // caused the exception. Otherwise this number will dominate the exceptions count.
             rpcServer.getMetrics().exception(sizeIOE);
           }
 
-          // Now that there's an exception is know to be created
+          // Now that there's an exception is known to be created
           // use it for the response.
           //
           // This will create a copy in the builder.
@@ -755,9 +760,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           } else {
             pbResult = ProtobufUtil.toResult(r);
           }
-          if (context != null) {
-            context.incrementResponseCellSize(Result.getTotalSizeOfCells(r));
-          }
+          lastBlock = addSize(context, r, lastBlock);
           resultOrExceptionBuilder =
             ClientProtos.ResultOrException.newBuilder().setResult(pbResult);
         }
@@ -1070,6 +1073,44 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return 0L;
   }
 
+  /**
+   * Method to account for the size of retained cells and retained data blocks.
+   * @return an object that represents the last referenced block from this response.
+   */
+  Object addSize(RpcCallContext context, Result r, Object lastBlock) {
+    if (context != null && !r.isEmpty()) {
+      for (Cell c : r.rawCells()) {
+        context.incrementResponseCellSize(CellUtil.estimatedHeapSizeOf(c));
+
+        // Since byte buffers can point all kinds of crazy places it's harder to keep track
+        // of which blocks are kept alive by what byte buffer.
+        // So we make a guess.
+        if (c instanceof ByteBufferedCell) {
+          ByteBufferedCell bbCell = (ByteBufferedCell) c;
+          ByteBuffer bb = bbCell.getValueByteBuffer();
+          if (bb != lastBlock) {
+            context.incrementResponseBlockSize(bb.capacity());
+            lastBlock = bb;
+          }
+        } else {
+          // We're using the last block being the same as the current block as
+          // a proxy for pointing to a new block. This won't be exact.
+          // If there are multiple gets that bounce back and forth
+          // Then it's possible that this will over count the size of
+          // referenced blocks. However it's better to over count and
+          // use two rpcs than to OOME the regionserver.
+          byte[] valueArray = c.getValueArray();
+          if (valueArray != lastBlock) {
+            context.incrementResponseBlockSize(valueArray.length);
+            lastBlock = valueArray;
+          }
+        }
+
+      }
+    }
+    return lastBlock;
+  }
+
   RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r)
       throws LeaseStillHeldException {
     Lease lease = regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
@@ -2467,6 +2508,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       assert scanner != null;
       RpcCallContext context = RpcServer.getCurrentCall();
+      Object lastBlock = null;
 
       quota = getQuotaManager().checkQuota(region, OperationQuota.OperationType.SCAN);
       long maxQuotaResultSize = Math.min(maxScannerResultSize, quota.getReadAvailable());
@@ -2500,11 +2542,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
               scanner, results, rows);
             if (!results.isEmpty()) {
               for (Result r : results) {
-                for (Cell cell : r.rawCells()) {
-                  if (context != null) {
-                    context.incrementResponseCellSize(CellUtil.estimatedSerializedSizeOf(cell));
-                  }
-                }
+                lastBlock = addSize(context, r, lastBlock);
               }
             }
             if (bypass != null && bypass.booleanValue()) {
@@ -2601,13 +2639,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
                   moreRows = scanner.nextRaw(values, scannerContext);
 
                   if (!values.isEmpty()) {
-                    for (Cell cell : values) {
-                      if (context != null) {
-                        context.incrementResponseCellSize(CellUtil.estimatedSerializedSizeOf(cell));
-                      }
-                    }
                     final boolean partial = scannerContext.partialResultFormed();
-                    results.add(Result.create(values, null, stale, partial));
+                    Result r = Result.create(values, null, stale, partial);
+                    lastBlock = addSize(context, r, lastBlock);
+                    results.add(r);
                     i++;
                   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/217036d8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
index 47dd7be..28e1855 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiRespectsLimits.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -36,6 +37,7 @@ import org.junit.experimental.categories.Category;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
 
 import static junit.framework.TestCase.assertEquals;
 
@@ -73,7 +75,7 @@ public class TestMultiRespectsLimits {
     TEST_UTIL.loadTable(t, FAMILY, false);
 
     // Split the table to make sure that the chunking happens accross regions.
-    try (final Admin admin = TEST_UTIL.getHBaseAdmin()) {
+    try (final Admin admin = TEST_UTIL.getAdmin()) {
       admin.split(name);
       TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
         @Override
@@ -87,16 +89,79 @@ public class TestMultiRespectsLimits {
     for (int i = 0; i < MAX_SIZE; i++) {
       gets.add(new Get(HBaseTestingUtility.ROWS[i]));
     }
-    Result[] results = t.get(gets);
-    assertEquals(MAX_SIZE, results.length);
+
     RpcServerInterface rpcServer = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer();
     BaseSource s = rpcServer.getMetrics().getMetricsSource();
+    long startingExceptions = METRICS_ASSERT.getCounter("exceptions", s);
+    long startingMultiExceptions = METRICS_ASSERT.getCounter("exceptions.multiResponseTooLarge", s);
+
+    Result[] results = t.get(gets);
+    assertEquals(MAX_SIZE, results.length);
 
     // Cells from TEST_UTIL.loadTable have a length of 27.
     // Multiplying by less than that gives an easy lower bound on size.
     // However in reality each kv is being reported as much higher than that.
-    METRICS_ASSERT.assertCounterGt("exceptions", (MAX_SIZE * 25) / MAX_SIZE, s);
+    METRICS_ASSERT.assertCounterGt("exceptions",
+        startingExceptions + ((MAX_SIZE * 25) / MAX_SIZE), s);
+    METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge",
+        startingMultiExceptions + ((MAX_SIZE * 25) / MAX_SIZE), s);
+  }
+
+  @Test
+  public void testBlockMultiLimits() throws Exception {
+    final TableName name = TableName.valueOf("testBlockMultiLimits");
+    Table t = TEST_UTIL.createTable(name, FAMILY);
+
+    final HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0);
+    RpcServerInterface rpcServer = regionServer.getRpcServer();
+    BaseSource s = rpcServer.getMetrics().getMetricsSource();
+    long startingExceptions = METRICS_ASSERT.getCounter("exceptions", s);
+    long startingMultiExceptions = METRICS_ASSERT.getCounter("exceptions.multiResponseTooLarge", s);
+
+    byte[] row = Bytes.toBytes("TEST");
+    byte[][] cols = new byte[][]{
+        Bytes.toBytes("0"), // Get this
+        Bytes.toBytes("1"), // Buffer
+        Bytes.toBytes("2"), // Get This
+        Bytes.toBytes("3"), // Buffer
+    };
+
+    // Set the value size so that one result will be less than the MAX_SIE
+    // however the block being reference will be larger than MAX_SIZE.
+    // This should cause the regionserver to try and send a result immediately.
+    byte[] value = new byte[MAX_SIZE - 200];
+    ThreadLocalRandom.current().nextBytes(value);
+
+    for (byte[] col:cols) {
+      Put p = new Put(row);
+      p.addImmutable(FAMILY, col, value);
+      t.put(p);
+    }
+
+    // Make sure that a flush happens
+    try (final Admin admin = TEST_UTIL.getAdmin()) {
+      admin.flush(name);
+      TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
+        @Override
+        public boolean evaluate() throws Exception {
+          return regionServer.getOnlineRegions(name).get(0).getMaxFlushedSeqId() > 3;
+        }
+      });
+    }
+
+    List<Get> gets = new ArrayList<>(2);
+    Get g0 = new Get(row);
+    g0.addColumn(FAMILY, cols[0]);
+    gets.add(g0);
+
+    Get g2 = new Get(row);
+    g2.addColumn(FAMILY, cols[2]);
+    gets.add(g2);
+
+    Result[] results = t.get(gets);
+    assertEquals(2, results.length);
+    METRICS_ASSERT.assertCounterGt("exceptions", startingExceptions, s);
     METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge",
-        (MAX_SIZE * 25) / MAX_SIZE, s);
+        startingMultiExceptions, s);
   }
 }


[20/43] hbase git commit: HBASE-14849 Add option to set block cache to false on SparkSQL executions (Zhan Zhang)

Posted by sy...@apache.org.
HBASE-14849 Add option to set block cache to false on SparkSQL executions (Zhan Zhang)


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

Branch: refs/heads/hbase-12439
Commit: e75e26e3c6cfe7fd378081839d60fc711c1e095f
Parents: 8c921ea
Author: tedyu <yu...@gmail.com>
Authored: Sat Dec 19 15:14:58 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sat Dec 19 15:14:58 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/spark/DefaultSource.scala      | 56 ++++++--------
 .../hadoop/hbase/spark/datasources/Bound.scala  |  2 -
 .../spark/datasources/HBaseSparkConf.scala      | 32 ++++++++
 .../spark/datasources/HBaseTableScanRDD.scala   | 14 ++--
 .../datasources/SerializableConfiguration.scala |  1 -
 .../hadoop/hbase/spark/DefaultSourceSuite.scala | 81 ++++++++------------
 .../hadoop/hbase/spark/HBaseTestSource.scala    | 63 +++++++++++++++
 7 files changed, 155 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
index 664cf57..73cab3c 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala
@@ -21,7 +21,9 @@ import java.util
 import java.util.concurrent.ConcurrentLinkedQueue
 
 import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.spark.datasources.{HBaseTableScanRDD, HBaseRegion, SerializableConfiguration}
+import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
+import org.apache.hadoop.hbase.spark.datasources.HBaseTableScanRDD
+import org.apache.hadoop.hbase.spark.datasources.SerializableConfiguration
 import org.apache.hadoop.hbase.types._
 import org.apache.hadoop.hbase.util.{Bytes, PositionedByteRange, SimplePositionedMutableByteRange}
 import org.apache.hadoop.hbase.{HBaseConfiguration, TableName}
@@ -49,8 +51,6 @@ class DefaultSource extends RelationProvider with Logging {
 
   val TABLE_KEY:String = "hbase.table"
   val SCHEMA_COLUMNS_MAPPING_KEY:String = "hbase.columns.mapping"
-  val BATCHING_NUM_KEY:String = "hbase.batching.num"
-  val CACHING_NUM_KEY:String = "hbase.caching.num"
   val HBASE_CONFIG_RESOURCES_LOCATIONS:String = "hbase.config.resources"
   val USE_HBASE_CONTEXT:String = "hbase.use.hbase.context"
   val PUSH_DOWN_COLUMN_FILTER:String = "hbase.push.down.column.filter"
@@ -71,35 +71,16 @@ class DefaultSource extends RelationProvider with Logging {
       new IllegalArgumentException("Invalid value for " + TABLE_KEY +" '" + tableName + "'")
 
     val schemaMappingString = parameters.getOrElse(SCHEMA_COLUMNS_MAPPING_KEY, "")
-    val batchingNumStr = parameters.getOrElse(BATCHING_NUM_KEY, "1000")
-    val cachingNumStr = parameters.getOrElse(CACHING_NUM_KEY, "1000")
     val hbaseConfigResources = parameters.getOrElse(HBASE_CONFIG_RESOURCES_LOCATIONS, "")
     val useHBaseReources = parameters.getOrElse(USE_HBASE_CONTEXT, "true")
     val usePushDownColumnFilter = parameters.getOrElse(PUSH_DOWN_COLUMN_FILTER, "true")
 
-    val batchingNum:Int = try {
-      batchingNumStr.toInt
-    } catch {
-      case e:NumberFormatException => throw
-        new IllegalArgumentException("Invalid value for " + BATCHING_NUM_KEY +" '"
-            + batchingNumStr + "'", e)
-    }
-
-    val cachingNum:Int = try {
-      cachingNumStr.toInt
-    } catch {
-      case e:NumberFormatException => throw
-        new IllegalArgumentException("Invalid value for " + CACHING_NUM_KEY +" '"
-            + cachingNumStr + "'", e)
-    }
-
     new HBaseRelation(tableName.get,
       generateSchemaMappingMap(schemaMappingString),
-      batchingNum.toInt,
-      cachingNum.toInt,
       hbaseConfigResources,
       useHBaseReources.equalsIgnoreCase("true"),
-      usePushDownColumnFilter.equalsIgnoreCase("true"))(sqlContext)
+      usePushDownColumnFilter.equalsIgnoreCase("true"),
+      parameters)(sqlContext)
   }
 
   /**
@@ -148,10 +129,6 @@ class DefaultSource extends RelationProvider with Logging {
  * @param tableName               HBase table that we plan to read from
  * @param schemaMappingDefinition SchemaMapping information to map HBase
  *                                Qualifiers to SparkSQL columns
- * @param batchingNum             The batching number to be applied to the
- *                                scan object
- * @param cachingNum              The caching number to be applied to the
- *                                scan object
  * @param configResources         Optional comma separated list of config resources
  *                                to get based on their URI
  * @param useHBaseContext         If true this will look to see if
@@ -162,14 +139,26 @@ class DefaultSource extends RelationProvider with Logging {
 case class HBaseRelation (val tableName:String,
                      val schemaMappingDefinition:
                      java.util.HashMap[String, SchemaQualifierDefinition],
-                     val batchingNum:Int,
-                     val cachingNum:Int,
                      val configResources:String,
                      val useHBaseContext:Boolean,
-                     val usePushDownColumnFilter:Boolean) (
+                     val usePushDownColumnFilter:Boolean,
+                     @transient parameters: Map[String, String] ) (
   @transient val sqlContext:SQLContext)
   extends BaseRelation with PrunedFilteredScan with Logging {
 
+  // The user supplied per table parameter will overwrite global ones in SparkConf
+  val blockCacheEnable = parameters.get(HBaseSparkConf.BLOCK_CACHE_ENABLE).map(_.toBoolean)
+    .getOrElse(
+      sqlContext.sparkContext.getConf.getBoolean(
+        HBaseSparkConf.BLOCK_CACHE_ENABLE, HBaseSparkConf.defaultBlockCacheEnable))
+  val cacheSize = parameters.get(HBaseSparkConf.CACHE_SIZE).map(_.toInt)
+    .getOrElse(
+      sqlContext.sparkContext.getConf.getInt(
+      HBaseSparkConf.CACHE_SIZE, HBaseSparkConf.defaultCachingSize))
+  val batchNum = parameters.get(HBaseSparkConf.BATCH_NUM).map(_.toInt)
+    .getOrElse(sqlContext.sparkContext.getConf.getInt(
+    HBaseSparkConf.BATCH_NUM,  HBaseSparkConf.defaultBatchNum))
+
   //create or get latest HBaseContext
   @transient val hbaseContext:HBaseContext = if (useHBaseContext) {
     LatestHBaseContextCache.latest
@@ -321,8 +310,9 @@ case class HBaseRelation (val tableName:String,
 
     if (resultRDD == null) {
       val scan = new Scan()
-      scan.setBatch(batchingNum)
-      scan.setCaching(cachingNum)
+      scan.setCacheBlocks(blockCacheEnable)
+      scan.setBatch(batchNum)
+      scan.setCaching(cacheSize)
       requiredQualifierDefinitionList.foreach( d =>
         scan.addColumn(d.columnFamilyBytes, d.qualifierBytes))
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
index c869f31..0f6098d 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/Bound.scala
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.hbase.spark.datasources
 
-import org.apache.hadoop.hbase.spark.SparkSQLPushDownFilter
-import org.apache.spark.Partition
 import org.apache.hadoop.hbase.spark.hbase._
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
new file mode 100644
index 0000000..67580b0
--- /dev/null
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.spark.datasources
+
+object HBaseSparkConf{
+  // This is the hbase configuration. User can either set them in SparkConf, which
+  // will take effect globally, or configure it per table, which will overwrite the value
+  // set in SparkConf. If not setted, the default value will take effect.
+  val BLOCK_CACHE_ENABLE = "spark.hbase.blockcache.enable"
+  // default block cache is set to true by default following hbase convention, but note that
+  // this potentially may slow down the system
+  val defaultBlockCacheEnable = true
+  val CACHE_SIZE = "spark.hbase.cacheSize"
+  val defaultCachingSize = 1000
+  val BATCH_NUM = "spark.hbase.batchNum"
+  val defaultBatchNum = 1000
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
index 958a986..eb9d39a 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
@@ -17,17 +17,11 @@
 
 package org.apache.hadoop.hbase.spark.datasources
 
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.hadoop.hbase.TableName
 import org.apache.hadoop.hbase.client._
-import org.apache.hadoop.hbase.filter.Filter
 import org.apache.hadoop.hbase.spark.{ScanRange, SchemaQualifierDefinition, HBaseRelation, SparkSQLPushDownFilter}
 import org.apache.hadoop.hbase.spark.hbase._
 import org.apache.hadoop.hbase.spark.datasources.HBaseResources._
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.{TaskContext, Logging, Partition}
+import org.apache.spark.{SparkEnv, TaskContext, Logging, Partition}
 import org.apache.spark.rdd.RDD
 
 import scala.collection.mutable
@@ -37,6 +31,7 @@ class HBaseTableScanRDD(relation: HBaseRelation,
      @transient val filter: Option[SparkSQLPushDownFilter] = None,
      val columns: Seq[SchemaQualifierDefinition] = Seq.empty
      )extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging  {
+  private def sparkConf = SparkEnv.get.conf
   var ranges = Seq.empty[Range]
   def addRange(r: ScanRange) = {
     val lower = if (r.lowerBound != null && r.lowerBound.length > 0) {
@@ -106,8 +101,9 @@ class HBaseTableScanRDD(relation: HBaseRelation,
         scan.addColumn(d.columnFamilyBytes, d.qualifierBytes)
       }
     }
-    scan.setBatch(relation.batchingNum)
-    scan.setCaching(relation.cachingNum)
+    scan.setCacheBlocks(relation.blockCacheEnable)
+    scan.setBatch(relation.batchNum)
+    scan.setCaching(relation.cacheSize)
     filter.foreach(scan.setFilter(_))
     scan
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
index 2452a52..42a5c32 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/SerializableConfiguration.scala
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.spark.datasources
 import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.spark.util.Utils
 
 import scala.util.control.NonFatal
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
index 2cee3a8..30ddfc4 100644
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
+++ b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala
@@ -18,10 +18,11 @@
 package org.apache.hadoop.hbase.spark
 
 import org.apache.hadoop.hbase.client.{Put, ConnectionFactory}
+import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
 import org.apache.hadoop.hbase.util.Bytes
 import org.apache.hadoop.hbase.{TableNotFoundException, TableName, HBaseTestingUtility}
 import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
 
 class DefaultSourceSuite extends FunSuite with
@@ -57,8 +58,11 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
     logInfo(" - creating table " + t2TableName)
     TEST_UTIL.createTable(TableName.valueOf(t2TableName), Bytes.toBytes(columnFamily))
     logInfo(" - created table")
-
-    sc = new SparkContext("local", "test")
+    val sparkConf = new SparkConf
+    sparkConf.set(HBaseSparkConf.BLOCK_CACHE_ENABLE, "true")
+    sparkConf.set(HBaseSparkConf.BATCH_NUM, "100")
+    sparkConf.set(HBaseSparkConf.CACHE_SIZE, "100")
+    sc  = new SparkContext("local", "test", sparkConf)
 
     val connection = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration)
     try {
@@ -139,18 +143,14 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
     df = sqlContext.load("org.apache.hadoop.hbase.spark",
       Map("hbase.columns.mapping" ->
         "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD STRING c:b,",
-        "hbase.table" -> "t1",
-        "hbase.batching.num" -> "100",
-        "cachingNum" -> "100"))
+        "hbase.table" -> "t1"))
 
     df.registerTempTable("hbaseTable1")
 
     df = sqlContext.load("org.apache.hadoop.hbase.spark",
       Map("hbase.columns.mapping" ->
         "KEY_FIELD INT :key, A_FIELD STRING c:a, B_FIELD STRING c:b,",
-        "hbase.table" -> "t2",
-        "hbase.batching.num" -> "100",
-        "cachingNum" -> "100"))
+        "hbase.table" -> "t2"))
 
     df.registerTempTable("hbaseTable2")
   }
@@ -635,49 +635,32 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging {
     }
   }
 
-  test("Test bad hbase.batching.num type") {
-    intercept[IllegalArgumentException] {
-      df = sqlContext.load("org.apache.hadoop.hbase.spark",
-        Map("hbase.columns.mapping" ->
-          "KEY_FIELD FOOBAR :key, A_FIELD STRING c:a, B_FIELD STRING c:b, I_FIELD STRING c:i,",
-          "hbase.table" -> "t1", "hbase.batching.num" -> "foo"))
-
-      df.registerTempTable("hbaseIntWrongTypeTmp")
-
-      val result = sqlContext.sql("SELECT KEY_FIELD, " +
-        "B_FIELD, I_FIELD FROM hbaseIntWrongTypeTmp")
-
-      assert(result.count() == 5)
-
-      val localResult = result.take(5)
-      localResult.length
-
-      val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll()
-      assert(executionRules.dynamicLogicExpression == null)
-
-
+  test("Test HBaseSparkConf matching") {
+    val df = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource",
+      Map("cacheSize" -> "100",
+        "batchNum" -> "100",
+        "blockCacheingEnable" -> "true", "rowNum" -> "10"))
+    assert(df.count() == 10)
+
+    val df1 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource",
+      Map("cacheSize" -> "1000",
+        "batchNum" -> "100", "blockCacheingEnable" -> "true", "rowNum" -> "10"))
+    intercept[Exception] {
+      assert(df1.count() == 10)
     }
-  }
-
-  test("Test bad hbase.caching.num type") {
-    intercept[IllegalArgumentException] {
-      df = sqlContext.load("org.apache.hadoop.hbase.spark",
-        Map("hbase.columns.mapping" ->
-          "KEY_FIELD FOOBAR :key, A_FIELD STRING c:a, B_FIELD STRING c:b, I_FIELD STRING c:i,",
-          "hbase.table" -> "t1", "hbase.caching.num" -> "foo"))
-
-      df.registerTempTable("hbaseIntWrongTypeTmp")
-
-      val result = sqlContext.sql("SELECT KEY_FIELD, B_FIELD, " +
-        "I_FIELD FROM hbaseIntWrongTypeTmp")
-
-      val localResult = result.take(10)
-      assert(localResult.length == 5)
-
-      val executionRules = DefaultSourceStaticUtils.lastFiveExecutionRules.poll()
-      assert(executionRules.dynamicLogicExpression == null)
 
+    val df2 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource",
+      Map("cacheSize" -> "100",
+        "batchNum" -> "1000", "blockCacheingEnable" -> "true", "rowNum" -> "10"))
+    intercept[Exception] {
+      assert(df2.count() == 10)
+    }
 
+    val df3 = sqlContext.load("org.apache.hadoop.hbase.spark.HBaseTestSource",
+      Map("cacheSize" -> "100",
+        "batchNum" -> "100", "blockCacheingEnable" -> "false", "rowNum" -> "10"))
+    intercept[Exception] {
+      assert(df3.count() == 10)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e75e26e3/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
new file mode 100644
index 0000000..83465d9
--- /dev/null
+++ b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/HBaseTestSource.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.spark
+
+import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
+import org.apache.spark.SparkEnv
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Row, SQLContext}
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types._
+
+class HBaseTestSource extends RelationProvider {
+  override def createRelation(
+      sqlContext: SQLContext,
+      parameters: Map[String, String]): BaseRelation = {
+    DummyScan(
+      parameters("cacheSize").toInt,
+      parameters("batchNum").toInt,
+      parameters("blockCacheingEnable").toBoolean,
+      parameters("rowNum").toInt)(sqlContext)
+  }
+}
+
+case class DummyScan(
+     cacheSize: Int,
+     batchNum: Int,
+     blockCachingEnable: Boolean,
+     rowNum: Int)(@transient val sqlContext: SQLContext)
+  extends BaseRelation with TableScan {
+  private def sparkConf = SparkEnv.get.conf
+  override def schema: StructType =
+    StructType(StructField("i", IntegerType, nullable = false) :: Nil)
+
+  override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(0 until rowNum)
+    .map(Row(_))
+    .map{ x =>
+      if (sparkConf.getInt(HBaseSparkConf.BATCH_NUM,
+        HBaseSparkConf.defaultBatchNum) != batchNum ||
+        sparkConf.getInt(HBaseSparkConf.CACHE_SIZE,
+          HBaseSparkConf.defaultCachingSize) != cacheSize ||
+        sparkConf.getBoolean(HBaseSparkConf.BLOCK_CACHE_ENABLE,
+          HBaseSparkConf.defaultBlockCacheEnable)
+          != blockCachingEnable) {
+        throw new Exception("HBase Spark configuration cannot be set properly")
+      }
+      x
+  }
+}


[39/43] hbase git commit: HBASE-15039 HMaster and RegionServers should try to refresh token keys from zk when facing InvalidToken (Yong Zhang)

Posted by sy...@apache.org.
HBASE-15039 HMaster and RegionServers should try to refresh token keys from zk when facing InvalidToken (Yong Zhang)


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

Branch: refs/heads/hbase-12439
Commit: 1c7e9ec124ab19d3c727d8294a8022d7b936bbfc
Parents: 3de0b04
Author: tedyu <yu...@gmail.com>
Authored: Fri Dec 25 10:32:51 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Dec 25 10:32:51 2015 -0800

----------------------------------------------------------------------
 .../token/AuthenticationTokenSecretManager.java | 21 +++++
 .../hbase/security/token/ZKSecretWatcher.java   | 25 +++++
 .../token/TestZKSecretWatcherRefreshKeys.java   | 99 ++++++++++++++++++++
 3 files changed, 145 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1c7e9ec1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
index 524b6f9..1f2bec4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/AuthenticationTokenSecretManager.java
@@ -143,6 +143,27 @@ public class AuthenticationTokenSecretManager
       throw new InvalidToken("Token has expired");
     }
     AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
+    if(masterKey == null) {
+      if(zkWatcher.getWatcher().isAborted()) {
+        LOG.error("ZookeeperWatcher is abort");
+        throw new InvalidToken("Token keys could not be sync from zookeeper"
+            + " because of ZookeeperWatcher abort");
+      }
+      synchronized (this) {
+        if (!leaderElector.isAlive() || leaderElector.isStopped()) {
+          LOG.warn("Thread leaderElector[" + leaderElector.getName() + ":"
+              + leaderElector.getId() + "] is stoped or not alive");
+          leaderElector.start();
+          LOG.info("Thread leaderElector [" + leaderElector.getName() + ":"
+              + leaderElector.getId() + "] is started");
+        }
+      }
+      zkWatcher.refreshKeys();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Sync token keys from zookeeper");
+      }
+      masterKey = allKeys.get(identifier.getKeyId());
+    }
     if (masterKey == null) {
       throw new InvalidToken("Unknown master key for token (id="+
           identifier.getKeyId()+")");

http://git-wip-us.apache.org/repos/asf/hbase/blob/1c7e9ec1/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
index 2264193..a1c3b66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/token/ZKSecretWatcher.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.security.token;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.IOException;
 import java.util.List;
 
@@ -211,4 +213,27 @@ public class ZKSecretWatcher extends ZooKeeperListener {
       watcher.abort("Failed serializing key "+key.getKeyId(), ioe);
     }
   }
+  
+  /**
+   * refresh keys
+   */
+  synchronized void refreshKeys() {
+    try {
+      List<ZKUtil.NodeAndData> nodes =
+          ZKUtil.getChildDataAndWatchForNewChildren(watcher, keysParentZNode);
+      refreshNodes(nodes);
+    } catch (KeeperException ke) {
+      LOG.fatal("Error reading data from zookeeper", ke);
+      watcher.abort("Error reading changed keys from zookeeper", ke);
+    }
+  }
+  
+  /**
+   * get token keys parent node
+   * @return token keys parent node
+   */
+  @VisibleForTesting
+  String getKeysParentZNode() {
+    return keysParentZNode;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1c7e9ec1/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
new file mode 100644
index 0000000..b239681
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestZKSecretWatcherRefreshKeys.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.token;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the refreshKeys in ZKSecretWatcher
+ */
+@Category({ SecurityTests.class, SmallTests.class })
+public class TestZKSecretWatcherRefreshKeys {
+  private static final Log LOG = LogFactory.getLog(TestZKSecretWatcherRefreshKeys.class);
+  private static HBaseTestingUtility TEST_UTIL;
+
+  private static class MockAbortable implements Abortable {
+    private boolean abort;
+    public void abort(String reason, Throwable e) {
+      LOG.info("Aborting: "+reason, e);
+      abort = true;
+    }
+
+    public boolean isAborted() {
+      return abort;
+    }
+  }
+  
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.startMiniZKCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  private static ZooKeeperWatcher newZK(Configuration conf, String name,
+      Abortable abort) throws Exception {
+    Configuration copy = HBaseConfiguration.create(conf);
+    ZooKeeperWatcher zk = new ZooKeeperWatcher(copy, name, abort);
+    return zk;
+  }
+
+  @Test
+  public void testRefreshKeys() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ZooKeeperWatcher zk = newZK(conf, "127.0.0.1", new MockAbortable());
+    AuthenticationTokenSecretManager keyManager = 
+        new AuthenticationTokenSecretManager(conf, zk, "127.0.0.1", 
+            60 * 60 * 1000, 60 * 1000);
+    ZKSecretWatcher watcher = new ZKSecretWatcher(conf, zk, keyManager);
+    ZKUtil.deleteChildrenRecursively(zk, watcher.getKeysParentZNode());
+    Integer[] keys = { 1, 2, 3, 4, 5, 6 };
+    for (Integer key : keys) {
+      AuthenticationKey ak = new AuthenticationKey(key,
+          System.currentTimeMillis() + 600 * 1000, null);
+      ZKUtil.createWithParents(zk,
+          ZKUtil.joinZNode(watcher.getKeysParentZNode(), key.toString()),
+          Writables.getBytes(ak));
+    }
+    Assert.assertNull(keyManager.getCurrentKey());
+    watcher.refreshKeys();
+    for (Integer key : keys) {
+      Assert.assertNotNull(keyManager.getKey(key.intValue()));
+    }
+  }
+}


[05/43] hbase git commit: HBASE-14991 Fix the '-feature' warning in scala build (Zhan Zhang)

Posted by sy...@apache.org.
HBASE-14991 Fix the '-feature' warning in scala build (Zhan Zhang)


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

Branch: refs/heads/hbase-12439
Commit: bbfff0d07266c5ed416f415c7f3cfecd54e0ae5b
Parents: 80fc18d
Author: Jonathan M Hsieh <jm...@apache.org>
Authored: Thu Dec 17 14:19:36 2015 -0800
Committer: Jonathan M Hsieh <jm...@apache.org>
Committed: Thu Dec 17 14:19:36 2015 -0800

----------------------------------------------------------------------
 hbase-spark/pom.xml                                               | 3 +++
 .../apache/hadoop/hbase/spark/datasources/HBaseResources.scala    | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bbfff0d0/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index ec911bd..2efbca0 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -537,6 +537,9 @@
                 <configuration>
                     <charset>${project.build.sourceEncoding}</charset>
                     <scalaVersion>${scala.version}</scalaVersion>
+                    <args>
+                        <arg>-feature</arg>
+                    </args>
                 </configuration>
                 <executions>
                     <execution>

http://git-wip-us.apache.org/repos/asf/hbase/blob/bbfff0d0/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
index 4417520..19a6ea7 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseResources.scala
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.spark.datasources
 import org.apache.hadoop.hbase.TableName
 import org.apache.hadoop.hbase.client._
 import org.apache.hadoop.hbase.spark.HBaseRelation
+import scala.language.implicitConversions
 
 // Resource and ReferencedResources are defined for extensibility,
 // e.g., consolidate scan and bulkGet in the future work.


[37/43] hbase git commit: HBASE-14822; addendum - handle callSeq.

Posted by sy...@apache.org.
HBASE-14822; addendum - handle callSeq.


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

Branch: refs/heads/hbase-12439
Commit: dfada43e90a0767518501f6878bf9896bed912ce
Parents: a1a19d9
Author: Lars Hofhansl <la...@apache.org>
Authored: Thu Dec 24 12:55:39 2015 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Thu Dec 24 12:55:39 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/regionserver/RSRpcServices.java  | 5 ++++-
 .../org/apache/hadoop/hbase/client/TestLeaseRenewal.java     | 8 ++++++++
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dfada43e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 75705e6..5d9b90f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -2518,9 +2518,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       }
       assert scanner != null;
       if (request.hasRenew() && request.getRenew()) {
+        rsh = scanners.get(scannerName);
         lease = regionServer.leases.removeLease(scannerName);
-        if (lease != null && scanners.containsKey(scannerName)) {
+        if (lease != null && rsh != null) {
           regionServer.leases.addLease(lease);
+          // Increment the nextCallSeq value which is the next expected from client.
+          rsh.incNextCallSeq();
         }
         return builder.build();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dfada43e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
index c89edf5..7170299 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestLeaseRenewal.java
@@ -26,10 +26,13 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.MetricsHBaseServerSource;
+import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
@@ -41,6 +44,8 @@ import org.junit.experimental.categories.Category;
 
 @Category(LargeTests.class)
 public class TestLeaseRenewal {
+  public MetricsAssertHelper HELPER = CompatibilityFactory.getInstance(MetricsAssertHelper.class);
+
   final Log LOG = LogFactory.getLog(getClass());
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte[] FAMILY = Bytes.toBytes("testFamily");
@@ -121,5 +126,8 @@ public class TestLeaseRenewal {
     assertFalse(rs.renewLease());
     rs.close();
     table.close();
+    MetricsHBaseServerSource serverSource = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
+        .getRpcServer().getMetrics().getMetricsSource();
+    HELPER.assertCounter("exceptions.OutOfOrderScannerNextException", 0, serverSource);
   }
 }


[07/43] hbase git commit: HBASE-14990 Tests in BaseTestHBaseFsck are run by its subclasses redundantly

Posted by sy...@apache.org.
HBASE-14990 Tests in BaseTestHBaseFsck are run by its subclasses redundantly


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

Branch: refs/heads/hbase-12439
Commit: 27160950caebd48b9ea45cdc579ace21d0cb47f6
Parents: 217036d
Author: chenheng <ch...@apache.org>
Authored: Fri Dec 18 11:37:58 2015 +0800
Committer: chenheng <ch...@apache.org>
Committed: Fri Dec 18 11:37:58 2015 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    | 301 -------------------
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   | 280 +++++++++++++++++
 2 files changed, 280 insertions(+), 301 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/27160950/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index a5a710f..bbb6b53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -18,14 +18,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -38,7 +31,6 @@ import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -53,7 +45,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -62,8 +53,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -73,28 +62,17 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
-import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
-import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
 import org.apache.zookeeper.KeeperException;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
 import org.junit.rules.TestName;
 
 /**
@@ -378,210 +356,6 @@ public class BaseTestHBaseFsck {
     return null;
   }
 
-
-
-
-  /**
-   * This creates and fixes a bad table with a missing region -- hole in meta
-   * and data present but .regioinfino missing (an orphan hdfs region)in the fs.
-   */
-  @Test (timeout=180000)
-  public void testHDFSRegioninfoMissing() throws Exception {
-    TableName table = TableName.valueOf("tableHDFSRegioninfoMissing");
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by leaving a hole in the meta data
-      admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
-          true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
-      admin.enableTable(table);
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck,
-          new ERROR_CODE[] { ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-              ERROR_CODE.HOLE_IN_REGION_CHAIN });
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(table).size());
-
-      // fix hole
-      doFsck(conf, true);
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf, false));
-      assertEquals(ROWKEYS.length, countRows());
-    } finally {
-      cleanupTable(table);
-    }
-  }
-
-  /**
-   * This creates and fixes a bad table with a region that is missing meta and
-   * not assigned to a region server.
-   */
-  @Test (timeout=180000)
-  public void testNotInMetaOrDeployedHole() throws Exception {
-    TableName table =
-        TableName.valueOf("tableNotInMetaOrDeployedHole");
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by leaving a hole in the meta data
-      admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
-          true, false); // don't rm from fs
-      admin.enableTable(table);
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck,
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(table).size());
-
-      // fix hole
-      assertErrors(doFsck(conf, true),
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf, false));
-      assertEquals(ROWKEYS.length, countRows());
-    } finally {
-      cleanupTable(table);
-    }
-  }
-
-  @Test (timeout=180000)
-  public void testCleanUpDaughtersNotInMetaAfterFailedSplit() throws Exception {
-    TableName table = TableName.valueOf("testCleanUpDaughtersNotInMetaAfterFailedSplit");
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    try {
-      HTableDescriptor desc = new HTableDescriptor(table);
-      desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
-      createTable(TEST_UTIL, desc, null);
-
-      tbl = connection.getTable(desc.getTableName());
-      for (int i = 0; i < 5; i++) {
-        Put p1 = new Put(("r" + i).getBytes());
-        p1.addColumn(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
-        tbl.put(p1);
-      }
-      admin.flush(desc.getTableName());
-      List<HRegion> regions = cluster.getRegions(desc.getTableName());
-      int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
-      HRegionServer regionServer = cluster.getRegionServer(serverWith);
-      cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
-      SplitTransactionImpl st = (SplitTransactionImpl)
-        new SplitTransactionFactory(TEST_UTIL.getConfiguration())
-          .create(regions.get(0), Bytes.toBytes("r3"));
-      st.prepare();
-      st.stepsBeforePONR(regionServer, regionServer, false);
-      AssignmentManager am = cluster.getMaster().getAssignmentManager();
-      Map<String, RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
-      for (RegionState state : regionsInTransition.values()) {
-        am.regionOffline(state.getRegion());
-      }
-      Map<HRegionInfo, ServerName> regionsMap = new HashMap<HRegionInfo, ServerName>();
-      regionsMap.put(regions.get(0).getRegionInfo(), regionServer.getServerName());
-      am.assign(regionsMap);
-      am.waitForAssignment(regions.get(0).getRegionInfo());
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(table).size());
-
-      // fix hole
-      assertErrors(
-          doFsck(conf, false, true, false, false, false, false, false, false, false, false, false,
-            null),
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-              ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf, false));
-      assertEquals(5, countRows());
-    } finally {
-      if (tbl != null) {
-        tbl.close();
-        tbl = null;
-      }
-      cleanupTable(table);
-    }
-  }
-
-  /**
-   * This creates fixes a bad table with a hole in meta.
-   */
-  @Test (timeout=180000)
-  public void testNotInMetaHole() throws Exception {
-    TableName table =
-        TableName.valueOf("tableNotInMetaHole");
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // Mess it up by leaving a hole in the meta data
-      admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
-          true, false); // don't rm from fs
-      admin.enableTable(table);
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck,
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(table).size());
-
-      // fix hole
-      assertErrors(doFsck(conf, true),
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf, false));
-      assertEquals(ROWKEYS.length, countRows());
-    } finally {
-      cleanupTable(table);
-    }
-  }
-
-  /**
-   * This creates and fixes a bad table with a region that is in meta but has
-   * no deployment or data hdfs
-   */
-  @Test (timeout=180000)
-  public void testNotInHdfs() throws Exception {
-    TableName table =
-        TableName.valueOf("tableNotInHdfs");
-    try {
-      setupTable(table);
-      assertEquals(ROWKEYS.length, countRows());
-
-      // make sure data in regions, if in wal only there is no data loss
-      admin.flush(table);
-
-      // Mess it up by leaving a hole in the hdfs data
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
-          false, true); // don't rm meta
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(table).size());
-
-      // fix hole
-      doFsck(conf, true);
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf,false));
-      assertEquals(ROWKEYS.length - 2, countRows());
-    } finally {
-      cleanupTable(table);
-    }
-  }
-
-
   public void deleteTableDir(TableName table) throws IOException {
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
@@ -591,10 +365,6 @@ public class BaseTestHBaseFsck {
     LOG.info("Deleted " + p + " sucessfully? " + success);
   }
 
-
-
-
-
   /**
    * We don't have an easy way to verify that a flush completed, so we loop until we find a
    * legitimate hfile and return it.
@@ -701,69 +471,6 @@ public class BaseTestHBaseFsck {
     }
   }
 
-  /**
-   * This creates a table and simulates the race situation where a concurrent compaction or split
-   * has removed an colfam dir before the corruption checker got to it.
-   */
-  // Disabled because fails sporadically.  Is this test right?  Timing-wise, there could be no
-  // files in a column family on initial creation -- as suggested by Matteo.
-  @Ignore @Test(timeout=180000)
-  public void testQuarantineMissingFamdir() throws Exception {
-    TableName table = TableName.valueOf(name.getMethodName());
-    // inject a fault in the hfcc created.
-    final FileSystem fs = FileSystem.get(conf);
-    HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
-      @Override
-      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
-          throws IOException {
-        return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
-          AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
-          @Override
-          protected void checkColFamDir(Path p) throws IOException {
-            if (attemptedFirstHFile.compareAndSet(false, true)) {
-              assertTrue(fs.delete(p, true)); // make sure delete happened.
-            }
-            super.checkColFamDir(p);
-          }
-        };
-      }
-    };
-    doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
-    hbck.close();
-  }
-
-  /**
-   * This creates a table and simulates the race situation where a concurrent compaction or split
-   * has removed a region dir before the corruption checker got to it.
-   */
-  @Test(timeout=180000)
-  public void testQuarantineMissingRegionDir() throws Exception {
-    TableName table = TableName.valueOf(name.getMethodName());
-    // inject a fault in the hfcc created.
-    final FileSystem fs = FileSystem.get(conf);
-    HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
-      @Override
-      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
-      throws IOException {
-        return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
-          AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
-          @Override
-          protected void checkRegionDir(Path p) throws IOException {
-            if (attemptedFirstHFile.compareAndSet(false, true)) {
-              assertTrue(fs.delete(p, true)); // make sure delete happened.
-            }
-            super.checkRegionDir(p);
-          }
-        };
-      }
-    };
-    doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
-    hbck.close();
-  }
-
-
-
-
 
   static class MockErrorReporter implements ErrorReporter {
     static int calledCount = 0;
@@ -880,17 +587,9 @@ public class BaseTestHBaseFsck {
     }
   }
 
-
-
-
-
-
-
   @org.junit.Rule
   public TestName name = new TestName();
 
-
-
   public static class MasterSyncObserver extends BaseMasterObserver {
     volatile CountDownLatch tableCreationLatch = null;
     volatile CountDownLatch tableDeletionLatch = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/27160950/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index c1c49e2..21935f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -46,10 +46,13 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
 import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
+import org.apache.hadoop.hbase.regionserver.SplitTransactionFactory;
 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
@@ -62,6 +65,7 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -69,6 +73,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -1531,4 +1537,278 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
     zkw.close();
     replicationAdmin.close();
   }
+
+  /**
+   * This creates and fixes a bad table with a missing region -- hole in meta
+   * and data present but .regioinfino missing (an orphan hdfs region)in the fs.
+   */
+  @Test(timeout=180000)
+  public void testHDFSRegioninfoMissing() throws Exception {
+    TableName table = TableName.valueOf("tableHDFSRegioninfoMissing");
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // Mess it up by leaving a hole in the meta data
+      admin.disableTable(table);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+        true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+      admin.enableTable(table);
+
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck,
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION,
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
+      // holes are separate from overlap groups
+      assertEquals(0, hbck.getOverlapGroups(table).size());
+
+      // fix hole
+      doFsck(conf, true);
+
+      // check that hole fixed
+      assertNoErrors(doFsck(conf, false));
+      assertEquals(ROWKEYS.length, countRows());
+    } finally {
+      cleanupTable(table);
+    }
+  }
+
+  /**
+   * This creates and fixes a bad table with a region that is missing meta and
+   * not assigned to a region server.
+   */
+  @Test (timeout=180000)
+  public void testNotInMetaOrDeployedHole() throws Exception {
+    TableName table =
+      TableName.valueOf("tableNotInMetaOrDeployedHole");
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // Mess it up by leaving a hole in the meta data
+      admin.disableTable(table);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+        true, false); // don't rm from fs
+      admin.enableTable(table);
+
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck,
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
+      // holes are separate from overlap groups
+      assertEquals(0, hbck.getOverlapGroups(table).size());
+
+      // fix hole
+      assertErrors(doFsck(conf, true),
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
+
+      // check that hole fixed
+      assertNoErrors(doFsck(conf, false));
+      assertEquals(ROWKEYS.length, countRows());
+    } finally {
+      cleanupTable(table);
+    }
+  }
+
+  @Test (timeout=180000)
+  public void testCleanUpDaughtersNotInMetaAfterFailedSplit() throws Exception {
+    TableName table = TableName.valueOf("testCleanUpDaughtersNotInMetaAfterFailedSplit");
+    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+    try {
+      HTableDescriptor desc = new HTableDescriptor(table);
+      desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
+      createTable(TEST_UTIL, desc, null);
+
+      tbl = connection.getTable(desc.getTableName());
+      for (int i = 0; i < 5; i++) {
+        Put p1 = new Put(("r" + i).getBytes());
+        p1.addColumn(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
+        tbl.put(p1);
+      }
+      admin.flush(desc.getTableName());
+      List<HRegion> regions = cluster.getRegions(desc.getTableName());
+      int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
+      HRegionServer regionServer = cluster.getRegionServer(serverWith);
+      cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
+      SplitTransactionImpl st = (SplitTransactionImpl)
+        new SplitTransactionFactory(TEST_UTIL.getConfiguration())
+          .create(regions.get(0), Bytes.toBytes("r3"));
+      st.prepare();
+      st.stepsBeforePONR(regionServer, regionServer, false);
+      AssignmentManager am = cluster.getMaster().getAssignmentManager();
+      Map<String, RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
+      for (RegionState state : regionsInTransition.values()) {
+        am.regionOffline(state.getRegion());
+      }
+      Map<HRegionInfo, ServerName> regionsMap = new HashMap<HRegionInfo, ServerName>();
+      regionsMap.put(regions.get(0).getRegionInfo(), regionServer.getServerName());
+      am.assign(regionsMap);
+      am.waitForAssignment(regions.get(0).getRegionInfo());
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+        HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+        HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
+      // holes are separate from overlap groups
+      assertEquals(0, hbck.getOverlapGroups(table).size());
+
+      // fix hole
+      assertErrors(
+        doFsck(conf, false, true, false, false, false, false, false, false, false, false, false,
+          null),
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
+
+      // check that hole fixed
+      assertNoErrors(doFsck(conf, false));
+      assertEquals(5, countRows());
+    } finally {
+      if (tbl != null) {
+        tbl.close();
+        tbl = null;
+      }
+      cleanupTable(table);
+    }
+  }
+
+  /**
+   * This creates fixes a bad table with a hole in meta.
+   */
+  @Test (timeout=180000)
+  public void testNotInMetaHole() throws Exception {
+    TableName table =
+      TableName.valueOf("tableNotInMetaHole");
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // Mess it up by leaving a hole in the meta data
+      admin.disableTable(table);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+        true, false); // don't rm from fs
+      admin.enableTable(table);
+
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck,
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
+      // holes are separate from overlap groups
+      assertEquals(0, hbck.getOverlapGroups(table).size());
+
+      // fix hole
+      assertErrors(doFsck(conf, true),
+        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
+
+      // check that hole fixed
+      assertNoErrors(doFsck(conf, false));
+      assertEquals(ROWKEYS.length, countRows());
+    } finally {
+      cleanupTable(table);
+    }
+  }
+
+  /**
+   * This creates and fixes a bad table with a region that is in meta but has
+   * no deployment or data hdfs
+   */
+  @Test (timeout=180000)
+  public void testNotInHdfs() throws Exception {
+    TableName table =
+      TableName.valueOf("tableNotInHdfs");
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // make sure data in regions, if in wal only there is no data loss
+      admin.flush(table);
+
+      // Mess it up by leaving a hole in the hdfs data
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+        false, true); // don't rm meta
+
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
+        HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_HDFS});
+      // holes are separate from overlap groups
+      assertEquals(0, hbck.getOverlapGroups(table).size());
+
+      // fix hole
+      doFsck(conf, true);
+
+      // check that hole fixed
+      assertNoErrors(doFsck(conf,false));
+      assertEquals(ROWKEYS.length - 2, countRows());
+    } finally {
+      cleanupTable(table);
+    }
+  }
+
+  /**
+   * This creates a table and simulates the race situation where a concurrent compaction or split
+   * has removed an colfam dir before the corruption checker got to it.
+   */
+  // Disabled because fails sporadically.  Is this test right?  Timing-wise, there could be no
+  // files in a column family on initial creation -- as suggested by Matteo.
+  @Ignore
+  @Test(timeout=180000)
+  public void testQuarantineMissingFamdir() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    // inject a fault in the hfcc created.
+    final FileSystem fs = FileSystem.get(conf);
+    HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
+      @Override
+      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
+        throws IOException {
+        return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
+          AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
+          @Override
+          protected void checkColFamDir(Path p) throws IOException {
+            if (attemptedFirstHFile.compareAndSet(false, true)) {
+              assertTrue(fs.delete(p, true)); // make sure delete happened.
+            }
+            super.checkColFamDir(p);
+          }
+        };
+      }
+    };
+    doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
+    hbck.close();
+  }
+
+  /**
+   * This creates a table and simulates the race situation where a concurrent compaction or split
+   * has removed a region dir before the corruption checker got to it.
+   */
+  @Test(timeout=180000)
+  public void testQuarantineMissingRegionDir() throws Exception {
+    TableName table = TableName.valueOf(name.getMethodName());
+    // inject a fault in the hfcc created.
+    final FileSystem fs = FileSystem.get(conf);
+    HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
+      @Override
+      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
+        throws IOException {
+        return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
+          AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
+          @Override
+          protected void checkRegionDir(Path p) throws IOException {
+            if (attemptedFirstHFile.compareAndSet(false, true)) {
+              assertTrue(fs.delete(p, true)); // make sure delete happened.
+            }
+            super.checkRegionDir(p);
+          }
+        };
+      }
+    };
+    doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
+    hbck.close();
+  }
 }


[30/43] hbase git commit: HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implemenations (Ashish Singhi)

Posted by sy...@apache.org.
HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implemenations (Ashish Singhi)


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

Branch: refs/heads/hbase-12439
Commit: e00a04df10de70b029a2d1f115f97f9d79a05c6a
Parents: 1af98f2
Author: stack <st...@apache.org>
Authored: Wed Dec 23 07:29:18 2015 -0800
Committer: stack <st...@apache.org>
Committed: Wed Dec 23 07:29:18 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/AbstractRpcClient.java     | 39 ++++++++++++++++++--
 .../apache/hadoop/hbase/ipc/AsyncRpcClient.java |  9 ++---
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  | 31 ----------------
 3 files changed, 38 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e00a04df/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 6f5e78a..e33ef3a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -24,6 +24,13 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
+
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.SocketTimeoutException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +41,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
+import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -41,10 +49,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PoolMap;
 import org.apache.hadoop.io.compress.CompressionCodec;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-
 /**
  * Provides the basics for a RpcClient implementation like configuration and Logging.
  */
@@ -258,6 +262,33 @@ public abstract class AbstractRpcClient implements RpcClient {
   }
 
   /**
+   * Takes an Exception and the address we were trying to connect to and return an IOException with
+   * the input exception as the cause. The new exception provides the stack trace of the place where
+   * the exception is thrown and some extra diagnostics information. If the exception is
+   * ConnectException or SocketTimeoutException, return a new one of the same type; Otherwise return
+   * an IOException.
+   * @param addr target address
+   * @param exception the relevant exception
+   * @return an exception to throw
+   */
+  protected IOException wrapException(InetSocketAddress addr, Exception exception) {
+    if (exception instanceof ConnectException) {
+      // connection refused; include the host:port in the error
+      return (ConnectException) new ConnectException("Call to " + addr
+          + " failed on connection exception: " + exception).initCause(exception);
+    } else if (exception instanceof SocketTimeoutException) {
+      return (SocketTimeoutException) new SocketTimeoutException("Call to " + addr
+          + " failed because " + exception).initCause(exception);
+    } else if (exception instanceof ConnectionClosingException) {
+      return (ConnectionClosingException) new ConnectionClosingException("Call to " + addr
+          + " failed on local exception: " + exception).initCause(exception);
+    } else {
+      return (IOException) new IOException("Call to " + addr + " failed on local exception: "
+          + exception).initCause(exception);
+    }
+  }
+
+  /**
    * Blocking rpc channel that goes via hbase rpc.
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/e00a04df/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
index f972d0e..5c83fd1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
@@ -248,13 +248,10 @@ public class AsyncRpcClient extends AbstractRpcClient {
       Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get();
       return new Pair<>(response, pcrc.cellScanner());
     } catch (ExecutionException e) {
-      if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw new IOException(e.getCause());
-      }
+      throw wrapException(addr, e);
     } catch (TimeoutException e) {
-      throw new CallTimeoutException(promise.toString());
+      CallTimeoutException cte = new CallTimeoutException(promise.toString());
+      throw wrapException(addr, cte);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e00a04df/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
index d7a0029..b87e7dd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
@@ -81,7 +81,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.io.OutputStream;
-import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -1267,36 +1266,6 @@ public class RpcClientImpl extends AbstractRpcClient {
 
 
   /**
-   * Take an IOException and the address we were trying to connect to
-   * and return an IOException with the input exception as the cause.
-   * The new exception provides the stack trace of the place where
-   * the exception is thrown and some extra diagnostics information.
-   * If the exception is ConnectException or SocketTimeoutException,
-   * return a new one of the same type; Otherwise return an IOException.
-   *
-   * @param addr target address
-   * @param exception the relevant exception
-   * @return an exception to throw
-   */
-  protected IOException wrapException(InetSocketAddress addr,
-                                         IOException exception) {
-    if (exception instanceof ConnectException) {
-      //connection refused; include the host:port in the error
-      return (ConnectException)new ConnectException(
-         "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
-    } else if (exception instanceof SocketTimeoutException) {
-      return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
-        " failed because " + exception).initCause(exception);
-    } else if (exception instanceof ConnectionClosingException){
-      return (ConnectionClosingException) new ConnectionClosingException(
-          "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
-    } else {
-      return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
-        exception).initCause(exception);
-    }
-  }
-
-  /**
    * Interrupt the connections to the given ip:port server. This should be called if the server
    *  is known as actually dead. This will not prevent current operation to be retried, and,
    *  depending on their own behavior, they may retry on the same server. This can be a feature,


[32/43] hbase git commit: Revert "HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implemenations (Ashish Singhi)"

Posted by sy...@apache.org.
Revert "HBASE-15018 Inconsistent way of handling TimeoutException in the rpc client implemenations (Ashish Singhi)"

This reverts commit e00a04df10de70b029a2d1f115f97f9d79a05c6a.


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

Branch: refs/heads/hbase-12439
Commit: 04de427e57d144caf5a9cde3664dac780ed763ab
Parents: 8e0854c
Author: stack <st...@apache.org>
Authored: Wed Dec 23 15:31:10 2015 -0800
Committer: stack <st...@apache.org>
Committed: Wed Dec 23 15:31:10 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/AbstractRpcClient.java     | 39 ++------------------
 .../apache/hadoop/hbase/ipc/AsyncRpcClient.java |  9 +++--
 .../apache/hadoop/hbase/ipc/RpcClientImpl.java  | 31 ++++++++++++++++
 3 files changed, 41 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/04de427e/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index e33ef3a..6f5e78a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -24,13 +24,6 @@ import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
-
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -41,7 +34,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodec;
-import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -49,6 +41,10 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PoolMap;
 import org.apache.hadoop.io.compress.CompressionCodec;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
 /**
  * Provides the basics for a RpcClient implementation like configuration and Logging.
  */
@@ -262,33 +258,6 @@ public abstract class AbstractRpcClient implements RpcClient {
   }
 
   /**
-   * Takes an Exception and the address we were trying to connect to and return an IOException with
-   * the input exception as the cause. The new exception provides the stack trace of the place where
-   * the exception is thrown and some extra diagnostics information. If the exception is
-   * ConnectException or SocketTimeoutException, return a new one of the same type; Otherwise return
-   * an IOException.
-   * @param addr target address
-   * @param exception the relevant exception
-   * @return an exception to throw
-   */
-  protected IOException wrapException(InetSocketAddress addr, Exception exception) {
-    if (exception instanceof ConnectException) {
-      // connection refused; include the host:port in the error
-      return (ConnectException) new ConnectException("Call to " + addr
-          + " failed on connection exception: " + exception).initCause(exception);
-    } else if (exception instanceof SocketTimeoutException) {
-      return (SocketTimeoutException) new SocketTimeoutException("Call to " + addr
-          + " failed because " + exception).initCause(exception);
-    } else if (exception instanceof ConnectionClosingException) {
-      return (ConnectionClosingException) new ConnectionClosingException("Call to " + addr
-          + " failed on local exception: " + exception).initCause(exception);
-    } else {
-      return (IOException) new IOException("Call to " + addr + " failed on local exception: "
-          + exception).initCause(exception);
-    }
-  }
-
-  /**
    * Blocking rpc channel that goes via hbase rpc.
    */
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hbase/blob/04de427e/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
index 5c83fd1..f972d0e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcClient.java
@@ -248,10 +248,13 @@ public class AsyncRpcClient extends AbstractRpcClient {
       Message response = timeout > 0 ? promise.get(timeout, TimeUnit.MILLISECONDS) : promise.get();
       return new Pair<>(response, pcrc.cellScanner());
     } catch (ExecutionException e) {
-      throw wrapException(addr, e);
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      } else {
+        throw new IOException(e.getCause());
+      }
     } catch (TimeoutException e) {
-      CallTimeoutException cte = new CallTimeoutException(promise.toString());
-      throw wrapException(addr, cte);
+      throw new CallTimeoutException(promise.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04de427e/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
index b87e7dd..d7a0029 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java
@@ -81,6 +81,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InterruptedIOException;
 import java.io.OutputStream;
+import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketAddress;
@@ -1266,6 +1267,36 @@ public class RpcClientImpl extends AbstractRpcClient {
 
 
   /**
+   * Take an IOException and the address we were trying to connect to
+   * and return an IOException with the input exception as the cause.
+   * The new exception provides the stack trace of the place where
+   * the exception is thrown and some extra diagnostics information.
+   * If the exception is ConnectException or SocketTimeoutException,
+   * return a new one of the same type; Otherwise return an IOException.
+   *
+   * @param addr target address
+   * @param exception the relevant exception
+   * @return an exception to throw
+   */
+  protected IOException wrapException(InetSocketAddress addr,
+                                         IOException exception) {
+    if (exception instanceof ConnectException) {
+      //connection refused; include the host:port in the error
+      return (ConnectException)new ConnectException(
+         "Call to " + addr + " failed on connection exception: " + exception).initCause(exception);
+    } else if (exception instanceof SocketTimeoutException) {
+      return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
+        " failed because " + exception).initCause(exception);
+    } else if (exception instanceof ConnectionClosingException){
+      return (ConnectionClosingException) new ConnectionClosingException(
+          "Call to " + addr + " failed on local exception: " + exception).initCause(exception);
+    } else {
+      return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
+        exception).initCause(exception);
+    }
+  }
+
+  /**
    * Interrupt the connections to the given ip:port server. This should be called if the server
    *  is known as actually dead. This will not prevent current operation to be retried, and,
    *  depending on their own behavior, they may retry on the same server. This can be a feature,


[35/43] hbase git commit: HBASE-15032 hbase shell scan filter string assumes UTF-8 encoding (huaxiang sun)

Posted by sy...@apache.org.
HBASE-15032 hbase shell scan filter string assumes UTF-8 encoding (huaxiang sun)


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

Branch: refs/heads/hbase-12439
Commit: e15c48ed2cf025dd3b0790c55cdc4239cc0fc161
Parents: 0bdd6e4
Author: tedyu <yu...@gmail.com>
Authored: Thu Dec 24 07:00:22 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Thu Dec 24 07:00:22 2015 -0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/table.rb      |  6 ++++--
 hbase-shell/src/test/ruby/hbase/table_test.rb | 16 ++++++++++++++++
 2 files changed, 20 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e15c48ed/hbase-shell/src/main/ruby/hbase/table.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb
index 2535a68..b5769ca 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -359,7 +359,8 @@ EOF
       unless filter.class == String
         get.setFilter(filter)
       else
-        get.setFilter(org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter))
+        get.setFilter(
+          org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter.to_java_bytes))
       end
 
       get.setConsistency(org.apache.hadoop.hbase.client.Consistency.valueOf(consistency)) if consistency
@@ -458,7 +459,8 @@ EOF
         unless filter.class == String
           scan.setFilter(filter)
         else
-          scan.setFilter(org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter))
+          scan.setFilter(
+            org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter.to_java_bytes))
         end
 
         scan.setScanMetricsEnabled(enablemetrics) if enablemetrics

http://git-wip-us.apache.org/repos/asf/hbase/blob/e15c48ed/hbase-shell/src/test/ruby/hbase/table_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/table_test.rb b/hbase-shell/src/test/ruby/hbase/table_test.rb
index 70ed04b..d4547b7 100644
--- a/hbase-shell/src/test/ruby/hbase/table_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/table_test.rb
@@ -598,6 +598,22 @@ module Hbase
       end
     end
 
+    define_test "scan should support FILTER with non-ASCII bytes" do
+      @test_table.put(4, "x:a", "\x82")
+      begin
+        res = @test_table._scan_internal FILTER => "SingleColumnValueFilter('x', 'a', >=, 'binary:\x82', true, true)"
+        assert_not_equal(res, {}, "Result is empty")
+        assert_kind_of(Hash, res)
+        assert_not_nil(res['4'])
+        assert_not_nil(res['4']['x:a'])
+        assert_nil(res['1'])
+        assert_nil(res['2'])
+      ensure
+        # clean up newly added columns for this test only.
+        @test_table.delete(4, "x:a")
+      end
+    end
+
     define_test "scan hbase meta table" do
       res = table("hbase:meta")._scan_internal
       assert_not_nil(res)


[23/43] hbase git commit: HBASE-15022 replication_admin.rb throws undefined method `getZooKeeperClusterKey' for ZKUtil

Posted by sy...@apache.org.
HBASE-15022 replication_admin.rb throws undefined method `getZooKeeperClusterKey' for ZKUtil


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

Branch: refs/heads/hbase-12439
Commit: 9a297ef0a54de736bad53de3b0b518a9eec1865c
Parents: f56abf7
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Mon Dec 21 13:20:26 2015 -0800
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Mon Dec 21 13:20:26 2015 -0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/replication_admin.rb | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9a297ef0/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index 617073b..2a24829 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -22,7 +22,7 @@ include Java
 java_import org.apache.hadoop.hbase.client.replication.ReplicationAdmin
 java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig
 java_import org.apache.hadoop.hbase.util.Bytes
-java_import org.apache.hadoop.hbase.zookeeper.ZKUtil
+java_import org.apache.hadoop.hbase.zookeeper.ZKConfig
 java_import org.apache.hadoop.hbase.TableName
 
 # Wrapper for org.apache.hadoop.hbase.client.replication.ReplicationAdmin
@@ -62,7 +62,7 @@ module Hbase
 
         # Cluster Key is required for ReplicationPeerConfig for a custom replication endpoint
         if !endpoint_classname.nil? and cluster_key.nil?
-          cluster_key = ZKUtil.getZooKeeperClusterKey(@configuration)
+          cluster_key = ZKConfig.getZooKeeperClusterKey(@configuration)
         end
 
         # Optional parameters


[21/43] hbase git commit: Fix bad link found by Boris of vectorportal

Posted by sy...@apache.org.
Fix bad link found by Boris of vectorportal


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

Branch: refs/heads/hbase-12439
Commit: a7ba3110bb0bc40ba682523abb8aeaa0abb9c959
Parents: e75e26e
Author: stack <st...@apache.org>
Authored: Mon Dec 21 08:32:27 2015 -0800
Committer: stack <st...@apache.org>
Committed: Mon Dec 21 08:32:27 2015 -0800

----------------------------------------------------------------------
 src/main/site/asciidoc/sponsors.adoc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7ba3110/src/main/site/asciidoc/sponsors.adoc
----------------------------------------------------------------------
diff --git a/src/main/site/asciidoc/sponsors.adoc b/src/main/site/asciidoc/sponsors.adoc
index 3ad1e0e..4d7ebf3 100644
--- a/src/main/site/asciidoc/sponsors.adoc
+++ b/src/main/site/asciidoc/sponsors.adoc
@@ -29,7 +29,7 @@ The below companies have been gracious enough to provide their commerical tool o
 
 * link:http://www.yourkit.com[YourKit] allows us to use their link:http://www.yourkit.com/overview/index.jsp[Java Profiler].
 * Some of us use link:http://www.jetbrains.com/idea[IntelliJ IDEA] thanks to link:http://www.jetbrains.com/[JetBrains].
-* Thank you to Boris at link:http://www.vectorportal.com/idea[Vector Portal] for granting us a license on the image on which our logo is based.
+* Thank you to Boris at link:http://www.vectorportal.com/[Vector Portal] for granting us a license on the image on which our logo is based.
 
 == Sponsoring the Apache Software Foundation">
 To contribute to the Apache Software Foundation, a good idea in our opinion, see the link:http://www.apache.org/foundation/sponsorship.html[ASF Sponsorship] page.


[24/43] hbase git commit: HBASE-14654 Reenable TestMultiParallel#testActiveThreadsCount

Posted by sy...@apache.org.
HBASE-14654 Reenable TestMultiParallel#testActiveThreadsCount


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

Branch: refs/heads/hbase-12439
Commit: f018c371b574a2d51930ba99f441bd227a77ec23
Parents: 9a297ef
Author: chenheng <ch...@apache.org>
Authored: Tue Dec 22 11:33:25 2015 +0800
Committer: chenheng <ch...@apache.org>
Committed: Tue Dec 22 11:33:25 2015 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/client/HTable.java    | 5 +++--
 .../java/org/apache/hadoop/hbase/client/TestMultiParallel.java  | 3 ++-
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f018c371/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index d3f3bc4..ce5a44c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -128,14 +128,15 @@ public class HTable implements HTableInterface {
     if (maxThreads == 0) {
       maxThreads = 1; // is there a better default?
     }
+    int corePoolSize = conf.getInt("hbase.htable.threads.coresize", 1);
     long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60);
 
     // Using the "direct handoff" approach, new threads will only be created
     // if it is necessary and will grow unbounded. This could be bad but in HCM
     // we only create as many Runnables as there are region servers. It means
     // it also scales when new region servers are added.
-    ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, TimeUnit.SECONDS,
-        new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
+    ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime,
+      TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
     pool.allowCoreThreadTimeOut(true);
     return pool;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f018c371/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 59c863e..b2c5c11 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -144,8 +144,9 @@ public class TestMultiParallel {
    * @throws NoSuchFieldException
    * @throws SecurityException
    */
-  @Ignore ("Nice bug flakey... expected 5 but was 4..") @Test(timeout=300000)
+  @Test(timeout=300000)
   public void testActiveThreadsCount() throws Exception {
+    UTIL.getConfiguration().setLong("hbase.htable.threads.coresize", slaves + 1);
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
       ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration());
       try {


[33/43] hbase git commit: HBASE-14940 Make our unsafe based ops more safe.

Posted by sy...@apache.org.
HBASE-14940 Make our unsafe based ops more safe.


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

Branch: refs/heads/hbase-12439
Commit: 6fc2596ab37614fe35ccfebda0564e4721bd4b95
Parents: 04de427
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Dec 24 07:54:13 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Dec 24 07:54:13 2015 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/filter/FuzzyRowFilter.java     |  6 +--
 .../apache/hadoop/hbase/nio/SingleByteBuff.java |  7 +--
 .../hadoop/hbase/util/ByteBufferUtils.java      | 43 ++++++++++--------
 .../org/apache/hadoop/hbase/util/Bytes.java     | 14 +++---
 .../apache/hadoop/hbase/util/UnsafeAccess.java  | 46 +++++++++++++++++---
 5 files changed, 79 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6fc2596a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index a4c6c3b..e31f0b5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -94,7 +94,7 @@ public class FuzzyRowFilter extends FilterBase {
   }
 
   private void preprocessSearchKey(Pair<byte[], byte[]> p) {
-    if (UnsafeAccess.isAvailable() == false) {
+    if (UnsafeAccess.unaligned() == false) {
       // do nothing
       return;
     }
@@ -113,7 +113,7 @@ public class FuzzyRowFilter extends FilterBase {
    * @return mask array
    */
   private byte[] preprocessMask(byte[] mask) {
-    if (UnsafeAccess.isAvailable() == false) {
+    if (UnsafeAccess.unaligned() == false) {
       // do nothing
       return mask;
     }
@@ -322,7 +322,7 @@ public class FuzzyRowFilter extends FilterBase {
   static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length,
       byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
 
-    if (UnsafeAccess.isAvailable() == false) {
+    if (UnsafeAccess.unaligned() == false) {
       return satisfiesNoUnsafe(reverse, row, offset, length, fuzzyKeyBytes, fuzzyKeyMeta);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fc2596a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
index 6d71eb2..227216a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/nio/SingleByteBuff.java
@@ -34,6 +34,7 @@ import sun.nio.ch.DirectBuffer;
 public class SingleByteBuff extends ByteBuff {
 
   private static final boolean UNSAFE_AVAIL = UnsafeAccess.isAvailable();
+  private static final boolean UNSAFE_UNALIGNED = UnsafeAccess.unaligned();
 
   // Underlying BB
   private final ByteBuffer buf;
@@ -237,7 +238,7 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public short getShort(int index) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toShort(unsafeRef, unsafeOffset + index);
     }
     return this.buf.getShort(index);
@@ -261,7 +262,7 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public int getInt(int index) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toInt(unsafeRef, unsafeOffset + index);
     }
     return this.buf.getInt(index);
@@ -285,7 +286,7 @@ public class SingleByteBuff extends ByteBuff {
 
   @Override
   public long getLong(int index) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toLong(unsafeRef, unsafeOffset + index);
     }
     return this.buf.getLong(index);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fc2596a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 24105ab..7bcc872 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -47,6 +47,7 @@ public final class ByteBufferUtils {
   public final static int NEXT_BIT_SHIFT = 7;
   public final static int NEXT_BIT_MASK = 1 << 7;
   private static final boolean UNSAFE_AVAIL = UnsafeAccess.isAvailable();
+  private static final boolean UNSAFE_UNALIGNED = UnsafeAccess.unaligned();
 
   private ByteBufferUtils() {
   }
@@ -392,9 +393,12 @@ public final class ByteBufferUtils {
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
     } else {
-      for (int i = 0; i < length; ++i) {
-        putByte(out, destinationOffset + i, toByte(in, sourceOffset + i));
-      }
+      int outOldPos = out.position();
+      out.position(destinationOffset);
+      ByteBuffer inDup = in.duplicate();
+      inDup.position(sourceOffset).limit(sourceOffset + length);
+      out.put(inDup);
+      out.position(outOldPos);
     }
     return destinationOffset + length;
   }
@@ -414,15 +418,15 @@ public final class ByteBufferUtils {
     if (in.hasArray() && out.hasArray()) {
       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.position()
           + out.arrayOffset(), length);
+      skip(out, length);
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, sourceOffset, out, out.position(), length);
+      skip(out, length);
     } else {
-      int destOffset = out.position();
-      for (int i = 0; i < length; ++i) {
-        putByte(out, destOffset + i, toByte(in, sourceOffset + i));
-      }
+      ByteBuffer inDup = in.duplicate();
+      inDup.position(sourceOffset).limit(sourceOffset + length);
+      out.put(inDup);
     }
-    skip(out, length);
   }
 
   /**
@@ -574,7 +578,7 @@ public final class ByteBufferUtils {
   }
 
   public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       long offset1Adj, offset2Adj;
       Object refObj1 = null, refObj2 = null;
       if (buf1.isDirect()) {
@@ -612,7 +616,7 @@ public final class ByteBufferUtils {
   }
 
   public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       long offset1Adj;
       Object refObj1 = null;
       if (buf1.isDirect()) {
@@ -725,7 +729,7 @@ public final class ByteBufferUtils {
    * @return short value at offset
    */
   public static short toShort(ByteBuffer buffer, int offset) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toShort(buffer, offset);
     } else {
       return buffer.getShort(offset);
@@ -739,7 +743,7 @@ public final class ByteBufferUtils {
    * @return int value at offset
    */
   public static int toInt(ByteBuffer buffer, int offset) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toInt(buffer, offset);
     } else {
       return buffer.getInt(offset);
@@ -753,7 +757,7 @@ public final class ByteBufferUtils {
    * @return long value at offset
    */
   public static long toLong(ByteBuffer buffer, int offset) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       return UnsafeAccess.toLong(buffer, offset);
     } else {
       return buffer.getLong(offset);
@@ -767,7 +771,7 @@ public final class ByteBufferUtils {
    * @param val int to write out
    */
   public static void putInt(ByteBuffer buffer, int val) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       int newPos = UnsafeAccess.putInt(buffer, buffer.position(), val);
       buffer.position(newPos);
     } else {
@@ -810,7 +814,7 @@ public final class ByteBufferUtils {
    * @param val short to write out
    */
   public static void putShort(ByteBuffer buffer, short val) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       int newPos = UnsafeAccess.putShort(buffer, buffer.position(), val);
       buffer.position(newPos);
     } else {
@@ -825,7 +829,7 @@ public final class ByteBufferUtils {
    * @param val long to write out
    */
   public static void putLong(ByteBuffer buffer, long val) {
-    if (UNSAFE_AVAIL) {
+    if (UNSAFE_UNALIGNED) {
       int newPos = UnsafeAccess.putLong(buffer, buffer.position(), val);
       buffer.position(newPos);
     } else {
@@ -870,9 +874,10 @@ public final class ByteBufferUtils {
     } else if (UNSAFE_AVAIL) {
       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
     } else {
-      for (int i = 0; i < length; i++) {
-        out[destinationOffset + i] = in.get(sourceOffset + i);
-      }
+      int oldPos = in.position();
+      in.position(sourceOffset);
+      in.get(out, destinationOffset, length);
+      in.position(oldPos);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fc2596a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
index 685e402..987f1e2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -789,7 +789,7 @@ public class Bytes implements Comparable<Bytes> {
     if (length != SIZEOF_LONG || offset + length > bytes.length) {
       throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_LONG);
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.toLong(bytes, offset);
     } else {
       long l = 0;
@@ -830,7 +830,7 @@ public class Bytes implements Comparable<Bytes> {
       throw new IllegalArgumentException("Not enough room to put a long at"
           + " offset " + offset + " in a " + bytes.length + " byte array");
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.putLong(bytes, offset, val);
     } else {
       for(int i = offset + 7; i > offset; i--) {
@@ -981,7 +981,7 @@ public class Bytes implements Comparable<Bytes> {
     if (length != SIZEOF_INT || offset + length > bytes.length) {
       throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_INT);
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.toInt(bytes, offset);
     } else {
       int n = 0;
@@ -1065,7 +1065,7 @@ public class Bytes implements Comparable<Bytes> {
       throw new IllegalArgumentException("Not enough room to put an int at"
           + " offset " + offset + " in a " + bytes.length + " byte array");
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.putInt(bytes, offset, val);
     } else {
       for(int i= offset + 3; i > offset; i--) {
@@ -1135,7 +1135,7 @@ public class Bytes implements Comparable<Bytes> {
     if (length != SIZEOF_SHORT || offset + length > bytes.length) {
       throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_SHORT);
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.toShort(bytes, offset);
     } else {
       short n = 0;
@@ -1173,7 +1173,7 @@ public class Bytes implements Comparable<Bytes> {
       throw new IllegalArgumentException("Not enough room to put a short at"
           + " offset " + offset + " in a " + bytes.length + " byte array");
     }
-    if (UnsafeAccess.isAvailable()) {
+    if (UnsafeAccess.unaligned()) {
       return UnsafeAccess.putShort(bytes, offset, val);
     } else {
       bytes[offset+1] = (byte) val;
@@ -1477,7 +1477,7 @@ public class Bytes implements Comparable<Bytes> {
 
       static final Unsafe theUnsafe;
       static {
-        if (UnsafeAccess.isAvailable()) {
+        if (UnsafeAccess.unaligned()) {
           theUnsafe = UnsafeAccess.theUnsafe;
         } else {
           // It doesn't matter what we throw;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fc2596a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
index fd79b80..e72c9f0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.util;
 
 import java.lang.reflect.Field;
+import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.security.AccessController;
@@ -38,6 +39,7 @@ public final class UnsafeAccess {
   private static final Log LOG = LogFactory.getLog(UnsafeAccess.class);
 
   static final Unsafe theUnsafe;
+  private static boolean unaligned;
 
   /** The offset to the first element in a byte array. */
   public static final long BYTE_ARRAY_BASE_OFFSET;
@@ -45,6 +47,11 @@ public final class UnsafeAccess {
   static final boolean littleEndian = ByteOrder.nativeOrder()
       .equals(ByteOrder.LITTLE_ENDIAN);
 
+  // This number limits the number of bytes to copy per call to Unsafe's
+  // copyMemory method. A limit is imposed to allow for safepoint polling
+  // during a large copy
+  static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L;
+
   static {
     theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction<Object>() {
       @Override
@@ -60,22 +67,40 @@ public final class UnsafeAccess {
       }
     });
 
-    if(theUnsafe != null){
+    if (theUnsafe != null) {
       BYTE_ARRAY_BASE_OFFSET = theUnsafe.arrayBaseOffset(byte[].class);
+      try {
+        // Using java.nio.Bits#unaligned() to check for unaligned-access capability
+        Class<?> clazz = Class.forName("java.nio.Bits");
+        Method m = clazz.getDeclaredMethod("unaligned");
+        m.setAccessible(true);
+        unaligned = (boolean) m.invoke(null);
+      } catch (Exception e) {
+        unaligned = false;
+      }
     } else{
       BYTE_ARRAY_BASE_OFFSET = -1;
+      unaligned = false;
     }
   }
 
   private UnsafeAccess(){}
 
   /**
-   * @return true when the running JVM is having sun's Unsafe package available in it.
+   * @return true when running JVM is having sun's Unsafe package available in it.
    */
   public static boolean isAvailable() {
     return theUnsafe != null;
   }
 
+  /**
+   * @return true when running JVM is having sun's Unsafe package available in it and underlying
+   *         system having unaligned-access capability.
+   */
+  public static boolean unaligned() {
+    return unaligned;
+  }
+
   // APIs to read primitive data from a byte[] using Unsafe way
   /**
    * Converts a byte array to a short value considering it was written in big-endian format.
@@ -330,7 +355,17 @@ public final class UnsafeAccess {
       destBase = dest.array();
     }
     long srcAddress = srcOffset + BYTE_ARRAY_BASE_OFFSET;
-    theUnsafe.copyMemory(src, srcAddress, destBase, destAddress, length);
+    unsafeCopy(src, srcAddress, destBase, destAddress, length);
+  }
+
+  private static void unsafeCopy(Object src, long srcAddr, Object dst, long destAddr, long len) {
+    while (len > 0) {
+      long size = (len > UNSAFE_COPY_THRESHOLD) ? UNSAFE_COPY_THRESHOLD : len;
+      theUnsafe.copyMemory(src, srcAddr, dst, destAddr, len);
+      len -= size;
+      srcAddr += size;
+      destAddr += size;
+    }
   }
 
   /**
@@ -354,7 +389,7 @@ public final class UnsafeAccess {
       srcBase = src.array();
     }
     long destAddress = destOffset + BYTE_ARRAY_BASE_OFFSET;
-    theUnsafe.copyMemory(srcBase, srcAddress, dest, destAddress, length);
+    unsafeCopy(srcBase, srcAddress, dest, destAddress, length);
   }
 
   /**
@@ -383,8 +418,9 @@ public final class UnsafeAccess {
       destAddress = destOffset + BYTE_ARRAY_BASE_OFFSET + dest.arrayOffset();
       destBase = dest.array();
     }
-    theUnsafe.copyMemory(srcBase, srcAddress, destBase, destAddress, length);
+    unsafeCopy(srcBase, srcAddress, destBase, destAddress, length);
   }
+
   // APIs to add primitives to BBs
   /**
    * Put a short value out to the specified BB position in big-endian format.


[12/43] hbase git commit: HBASE-14980 Project Astro

Posted by sy...@apache.org.
HBASE-14980 Project Astro


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

Branch: refs/heads/hbase-12439
Commit: 58342a85fa77b0d415889dd1994c55c60525b83b
Parents: f8eab44
Author: stack <st...@apache.org>
Authored: Fri Dec 18 08:58:44 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Dec 18 08:59:01 2015 -0800

----------------------------------------------------------------------
 src/main/site/xdoc/poweredbyhbase.xml | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/58342a85/src/main/site/xdoc/poweredbyhbase.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/poweredbyhbase.xml b/src/main/site/xdoc/poweredbyhbase.xml
index 273e064..c940109 100644
--- a/src/main/site/xdoc/poweredbyhbase.xml
+++ b/src/main/site/xdoc/poweredbyhbase.xml
@@ -42,6 +42,18 @@ under the License.
     write data to HBase and run mapreduce jobs to process then store it back to
     HBase or external systems. Our production cluster has been running since Oct 2008.</dd>
 
+  <dt><a href="http://huaweibigdata.github.io/astro/">Project Astro</a></dt>
+  <dd>
+    Astro provides fast Spark SQL/DataFrame capabilities to HBase data,
+    featuring super-efficient access to multi-dimensional HBase rows through
+    native Spark execution in HBase coprocessor plus systematic and accurate
+    partition pruning and predicate pushdown from arbitrarily complex data
+    filtering logic. The batch load is optimized to run on the Spark execution
+    engine. Note that <a href="http://spark-packages.org/package/Huawei-Spark/Spark-SQL-on-HBase">Spark-SQL-on-HBase</a>
+    is the release site. Interested parties are free to make clones and claim
+    to be "latest(and active)", but they are not endorsed by the owner.
+  </dd>
+
   <dt><a href="http://axibase.com/products/axibase-time-series-database/">Axibase
     Time Series Database (ATSD)</a></dt>
   <dd>ATSD runs on top of HBase to collect, analyze and visualize time series


[16/43] hbase git commit: Fix link to vectorportal.com noticed by Boris of vectorportal

Posted by sy...@apache.org.
Fix link to vectorportal.com noticed by Boris of vectorportal


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

Branch: refs/heads/hbase-12439
Commit: cdca22a36d0e2ec87cea732e0769c4fa4fa37e57
Parents: eb59d4d
Author: stack <st...@apache.org>
Authored: Fri Dec 18 14:14:57 2015 -0800
Committer: stack <st...@apache.org>
Committed: Fri Dec 18 14:14:57 2015 -0800

----------------------------------------------------------------------
 src/main/site/xdoc/sponsors.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cdca22a3/src/main/site/xdoc/sponsors.xml
----------------------------------------------------------------------
diff --git a/src/main/site/xdoc/sponsors.xml b/src/main/site/xdoc/sponsors.xml
index 6922c0c..332f56a 100644
--- a/src/main/site/xdoc/sponsors.xml
+++ b/src/main/site/xdoc/sponsors.xml
@@ -38,7 +38,7 @@ under the License.
         so we can untangle our interdependency mess.</li>
 	<li><a href="http://www.yourkit.com">YourKit</a> allows us to use their <a href="http://www.yourkit.com/overview/index.jsp">Java Profiler</a>.</li>
 	<li>Some of us use <a href="http://www.jetbrains.com/idea">IntelliJ IDEA</a> thanks to <a href="http://www.jetbrains.com/">JetBrains</a>.</li>
-    <li>Thank you to Boris at <a href="http://www.vectorportal.com/idea">Vector Portal</a> for granting us a license on the image on which our logo is based.</li>
+  <li>Thank you to Boris at <a href="http://www.vectorportal.com/">Vector Portal</a> for granting us a license on the <a href="http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp">image</a> on which our logo is based.</li>
 </ul>
 </p>
 </section>