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

[36/50] [abbrv] hbase git commit: REVERT of revert of "HBASE-16308 Contain protobuf references Gather up the pb references into a few locations only rather than have pb references distributed all about the code base." This is a revert of a revert; i.e. w

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
index 0cccce1..daa8942 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java
@@ -23,8 +23,6 @@ import static org.junit.Assert.assertTrue;
 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.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -43,7 +41,6 @@ import org.junit.experimental.categories.Category;
 
 @Category({MasterTests.class, SmallTests.class})
 public class TestRegionLocationFinder {
-  private static final Log LOG = LogFactory.getLog(TestRegionLocationFinder.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static MiniHBaseCluster cluster;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
index 5ae02e4..cec8a74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/mapreduce/TestMobSweepMapper.java
@@ -49,13 +49,11 @@ import org.mockito.stubbing.Answer;
 
 @Category(SmallTests.class)
 public class TestMobSweepMapper {
-
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
-    TEST_UTIL.startMiniCluster(1);
+    TEST_UTIL.startMiniCluster();
   }
 
   @AfterClass
@@ -93,7 +91,7 @@ public class TestMobSweepMapper {
     lock.acquire();
     try {
       Mapper<ImmutableBytesWritable, Result, Text, KeyValue>.Context ctx =
-        mock(Mapper.Context.class);
+          mock(Mapper.Context.class);
       when(ctx.getConfiguration()).thenReturn(configuration);
       SweepMapper map = new SweepMapper();
       doAnswer(new Answer<Void>() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 6e68201..848010b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -62,6 +62,7 @@ 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.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -198,19 +199,20 @@ public class TestHRegionServerBulkLoad {
       }
 
       // bulk load HFiles
-      final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      final ClusterConnection conn = (ClusterConnection)UTIL.getConnection();
       Table table = conn.getTable(tableName);
-      final String bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(conn);
-      RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+      final String bulkToken = new SecureBulkLoadClient(UTIL.getConfiguration(), table).
+          prepareBulkLoad(conn);
+      RegionServerCallable<Void> callable = new RegionServerCallable<Void>(conn,
+          new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        public Void rpcCall() throws Exception {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
               + Bytes.toStringBinary(getRow()));
           SecureBulkLoadClient secureClient = null;
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
-            secureClient = new SecureBulkLoadClient(table);
+            secureClient = new SecureBulkLoadClient(UTIL.getConfiguration(), table);
             secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName,
                   true, null, bulkToken);
           }
@@ -224,15 +226,15 @@ public class TestHRegionServerBulkLoad {
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn,
+            new RpcControllerFactory(UTIL.getConfiguration()), tableName, Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =
               conn.getAdmin(getLocation().getServerName());
-            CompactRegionRequest request =
-              RequestConverter.buildCompactRegionRequest(
+            CompactRegionRequest request = RequestConverter.buildCompactRegionRequest(
                 getLocation().getRegionInfo().getRegionName(), true, null);
             server.compactRegion(null, request);
             numCompactions.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
index d55adef..e5361a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+
 import com.google.common.collect.Lists;
 
 /**
@@ -89,10 +91,12 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
 
       // bulk load HFiles
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
         @Override
-        public Void call(int callTimeout) throws Exception {
+        protected Void rpcCall() throws Exception {
           LOG.info("Non-secure old client");
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
               BulkLoadHFileRequest request =
@@ -109,9 +113,10 @@ public class TestHRegionServerBulkLoadWithOldClient extends TestHRegionServerBul
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
index 6de6261..f337be5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java
@@ -33,13 +33,12 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.client.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -62,7 +61,8 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
     super(duration);
   }
 
-  private static final Log LOG = LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
+  private static final Log LOG =
+      LogFactory.getLog(TestHRegionServerBulkLoadWithOldSecureEndpoint.class);
 
   @BeforeClass
   public static void setUpBeforeClass() throws IOException {
@@ -103,16 +103,17 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection();
       Table table = conn.getTable(tableName);
       final String bulkToken = new SecureBulkLoadEndpointClient(table).prepareBulkLoad(tableName);
+      RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration());
       RegionServerCallable<Void> callable =
-          new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+          new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+              Bytes.toBytes("aaa")) {
             @Override
-            public Void call(int callTimeout) throws Exception {
-              LOG.debug("Going to connect to server " + getLocation() + " for row "
-                  + Bytes.toStringBinary(getRow()));
+            protected Void rpcCall() throws Exception {
+              LOG.debug("Going to connect to server " + getLocation() + " for row " +
+                  Bytes.toStringBinary(getRow()));
               try (Table table = conn.getTable(getTableName())) {
-                boolean loaded =
-                    new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null,
-                      bulkToken, getLocation().getRegionInfo().getStartKey());
+                boolean loaded = new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths,
+                    null, bulkToken, getLocation().getRegionInfo().getStartKey());
               }
               return null;
             }
@@ -124,9 +125,10 @@ public class TestHRegionServerBulkLoadWithOldSecureEndpoint extends TestHRegionS
       // Periodically do compaction to reduce the number of open file handles.
       if (numBulkLoads.get() % 5 == 0) {
         // 5 * 50 = 250 open file handles!
-        callable = new RegionServerCallable<Void>(conn, tableName, Bytes.toBytes("aaa")) {
+        callable = new RegionServerCallable<Void>(conn, rpcControllerFactory, tableName,
+            Bytes.toBytes("aaa")) {
           @Override
-          public Void call(int callTimeout) throws Exception {
+          protected Void rpcCall() throws Exception {
             LOG.debug("compacting " + getLocation() + " for row "
                 + Bytes.toStringBinary(getRow()));
             AdminProtos.AdminService.BlockingInterface server =

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
index b906e84..2d9ba6e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -33,6 +32,7 @@ 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.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.CellComparator;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/hbase/blob/45bb6180/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
index fa66d69..3e90fe1 100644
--- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
+++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.hbase.spark;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -37,6 +35,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.ByteString;
 
 /**
  * This filter will push down all qualifier logic given to us