Repository: hbase
Updated Branches:
  refs/heads/branch-2 e294dcd80 -> 0eab16fde


http://git-wip-us.apache.org/repos/asf/hbase/blob/0eab16fd/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
index 96d54a5..b168be7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import static org.junit.Assert.*;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
@@ -31,20 +29,21 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 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.regionserver.MiniBatchOperationInProgress;
-import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -54,7 +53,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 @Category(MediumTests.class)
 public class TestRegionObserverForAddingMutationsFromCoprocessors {
@@ -272,7 +272,7 @@ public class 
TestRegionObserverForAddingMutationsFromCoprocessors {
     static WALEdit savedEdit = null;
     @Override
     public void postWALWrite(ObserverContext<? extends 
WALCoprocessorEnvironment> ctx,
-        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+                             RegionInfo info, WALKey logKey, WALEdit logEdit) 
throws IOException {
       if 
(info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit")))
 {
         savedEdit = logEdit;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0eab16fd/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 979936f..c3627f7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -18,13 +18,6 @@
  */
 package org.apache.hadoop.hbase.namespace;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
@@ -54,6 +47,7 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -87,6 +81,13 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 @Category(MediumTests.class)
 public class TestNamespaceAuditor {
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
@@ -303,7 +304,7 @@ public class TestNamespaceAuditor {
     @Override
     public synchronized void preMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final HRegionInfo[] regionsToMerge) throws IOException {
+        final RegionInfo[] regionsToMerge) throws IOException {
       notifyAll();
       if (shouldFailMerge) {
         throw new IOException("fail merge");
@@ -540,7 +541,7 @@ public class TestNamespaceAuditor {
 
     @Override
     public void 
preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       if (throwExceptionInPreCreateTableAction) {
         throw new IOException("Throw exception as it is demanded.");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0eab16fd/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 1fee7ca..cfb0ce0 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -17,12 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import static 
org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
-import static 
org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.io.IOException;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -39,6 +33,7 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Table;
@@ -47,22 +42,23 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.WALObserver;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import 
org.apache.hadoop.hbase.replication.ReplicationEndpoint.ReplicateContext;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import 
org.apache.hadoop.hbase.replication.regionserver.RegionReplicaReplicationEndpoint.RegionReplicaReplayCallable;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -72,7 +68,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import static 
org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.closeRegion;
+import static 
org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster.openRegion;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Tests RegionReplicaReplicationEndpoint. Unlike 
TestRegionReplicaReplicationEndpoint this
@@ -154,7 +154,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     }
     @Override
     public void postWALWrite(ObserverContext<? extends 
WALCoprocessorEnvironment> ctx,
-        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+                             RegionInfo info, WALKey logKey, WALEdit logEdit) 
throws IOException {
       // only keep primary region's edits
       if (logKey.getTablename().equals(tableName) && info.getReplicaId() == 0) 
{
         entries.add(new Entry(logKey, logEdit));

http://git-wip-us.apache.org/repos/asf/hbase/blob/0eab16fd/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index 19cbc38..afb14fe 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.hbase.security.access;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedActionException;
@@ -30,30 +27,32 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 
+import com.google.protobuf.BlockingRpcChannel;
+import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
@@ -63,12 +62,12 @@ import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.ServiceException;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Utility methods for testing security
@@ -630,7 +629,7 @@ public class SecureTestUtil {
     @Override
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        TableDescriptor desc, HRegionInfo[] regions) throws IOException {
+        TableDescriptor desc, RegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0eab16fd/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 d1c287f..58ddc2d 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,9 +18,6 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -40,36 +37,37 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Admin;
 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.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
@@ -77,6 +75,9 @@ import 
org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
 import org.apache.zookeeper.KeeperException;
 import org.junit.rules.TestName;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 /**
  * This is the base class for  HBaseFsck's ability to detect reasons for 
inconsistent tables.
  *
@@ -601,7 +602,7 @@ public class BaseTestHBaseFsck {
     public void postCompletedCreateTableAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final TableDescriptor desc,
-        final HRegionInfo[] regions) throws IOException {
+        final RegionInfo[] regions) throws IOException {
       // the AccessController test, some times calls only and directly the
       // postCompletedCreateTableAction()
       if (tableCreationLatch != null) {

Reply via email to