diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 5abf6a43d68..108662ce57c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -190,7 +190,7 @@ public class MetaTableAccessor { // There should still be a way to use this method with an unmanaged connection. if (connection instanceof ClusterConnection) { if (((ClusterConnection) connection).isManaged()) { - return new HTable(TableName.META_TABLE_NAME, (ClusterConnection) connection); + return new HTable(TableName.META_TABLE_NAME, connection); } } return connection.getTable(TableName.META_TABLE_NAME); 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 3db8c1cf7b0..ea7dc60ea41 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 @@ -41,9 +41,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HConnection; -import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; @@ -91,7 +90,7 @@ public class ReplicationAdmin implements Closeable { public static final String REPLICATIONGLOBAL = Integer .toString(HConstants.REPLICATION_SCOPE_GLOBAL); - private final HConnection connection; + private final Connection connection; // TODO: replication should be managed by master. All the classes except ReplicationAdmin should // be moved to hbase-server. Resolve it in HBASE-11392. private final ReplicationQueuesClient replicationQueuesClient; @@ -109,7 +108,7 @@ public class ReplicationAdmin implements Closeable { throw new RuntimeException("hbase.replication isn't true, please " + "enable it in order to use replication"); } - this.connection = HConnectionManager.getConnection(conf); + this.connection = ConnectionFactory.createConnection(conf); ZooKeeperWatcher zkw = createZooKeeperWatcher(); try { this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection); @@ -323,7 +322,7 @@ public class ReplicationAdmin implements Closeable { * Append the replicable table-cf config of the specified peer * @param id a short that identifies the cluster * @param tableCfs table-cfs config str - * @throws KeeperException + * @throws ReplicationException */ public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException { appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs)); @@ -333,7 +332,7 @@ public class ReplicationAdmin implements Closeable { * Append the replicable table-cf config of the specified peer * @param id a short that identifies the cluster * @param tableCfs A map from tableName to column family names - * @throws KeeperException + * @throws ReplicationException */ public void appendPeerTableCFs(String id, Map> tableCfs) throws ReplicationException { @@ -469,7 +468,7 @@ public class ReplicationAdmin implements Closeable { public List> listReplicated() throws IOException { List> replicationColFams = new ArrayList>(); - Admin admin = new HBaseAdmin(this.connection.getConfiguration()); + Admin admin = connection.getAdmin(); HTableDescriptor[] tables; try { tables = admin.listTables(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index da643fcff28..43e0b756a73 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -145,7 +145,8 @@ public class TestClientNoCluster extends Configured implements Tool { Configuration localConfig = HBaseConfiguration.create(this.conf); // This override mocks up our exists/get call to throw a RegionServerStoppedException. localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); - Table table = new HTable(localConfig, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(localConfig); + Table table = connection.getTable(TableName.META_TABLE_NAME); Throwable t = null; LOG.info("Start"); try { @@ -161,6 +162,7 @@ public class TestClientNoCluster extends Configured implements Tool { } finally { table.close(); } + connection.close(); LOG.info("Stop"); assertTrue(t != null); } @@ -182,7 +184,8 @@ public class TestClientNoCluster extends Configured implements Tool { // and it has expired. Otherwise, if this functionality is broke, all retries will be run -- // all ten of them -- and we'll get the RetriesExhaustedException exception. localConfig.setInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT, pause - 1); - Table table = new HTable(localConfig, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(localConfig); + Table table = connection.getTable(TableName.META_TABLE_NAME); Throwable t = null; try { // An exists call turns into a get w/ a flag. @@ -196,6 +199,7 @@ public class TestClientNoCluster extends Configured implements Tool { fail(); } finally { table.close(); + connection.close(); } assertTrue(t != null); } @@ -216,7 +220,8 @@ public class TestClientNoCluster extends Configured implements Tool { // Go against meta else we will try to find first region for the table on construction which // means we'll have to do a bunch more mocking. Tests that go against meta only should be // good for a bit of testing. - Table table = new HTable(this.conf, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(this.conf); + Table table = connection.getTable(TableName.META_TABLE_NAME); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); try { Result result = null; @@ -226,6 +231,7 @@ public class TestClientNoCluster extends Configured implements Tool { } finally { scanner.close(); table.close(); + connection.close(); } } @@ -236,7 +242,8 @@ public class TestClientNoCluster extends Configured implements Tool { // Go against meta else we will try to find first region for the table on construction which // means we'll have to do a bunch more mocking. Tests that go against meta only should be // good for a bit of testing. - Table table = new HTable(this.conf, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(TableName.META_TABLE_NAME); ResultScanner scanner = table.getScanner(HConstants.CATALOG_FAMILY); try { Result result = null; @@ -246,6 +253,7 @@ public class TestClientNoCluster extends Configured implements Tool { } finally { scanner.close(); table.close(); + connection.close(); } } diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java index 87e655e80c0..511bf4669f4 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java @@ -125,7 +125,7 @@ public class TestBulkDeleteProtocol { private long invokeBulkDeleteProtocol(TableName tableName, final Scan scan, final int rowBatchSize, final DeleteType deleteType, final Long timeStamp) throws Throwable { - Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = TEST_UTIL.getConnection().getTable(tableName); long noOfDeletedRows = 0L; Batch.Call callable = new Batch.Call() { @@ -220,7 +220,7 @@ public class TestBulkDeleteProtocol { htd.addFamily(new HColumnDescriptor(FAMILY1)); htd.addFamily(new HColumnDescriptor(FAMILY2)); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = TEST_UTIL.getConnection().getTable(tableName); List puts = new ArrayList(100); for (int j = 0; j < 100; j++) { Put put = new Put(Bytes.toBytes(j)); @@ -430,7 +430,7 @@ public class TestBulkDeleteProtocol { hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here htd.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = TEST_UTIL.getConnection().getTable(tableName); return ht; } diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java index ddc58477550..481cb917cdf 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java @@ -71,7 +71,7 @@ public class TestRowCountEndpoint { // @Ignore @Test public void testEndpoint() throws Throwable { - Table table = new HTable(CONF, TEST_TABLE); + Table table = TEST_UTIL.getConnection().getTable(TEST_TABLE); // insert some test rows for (int i=0; i<5; i++) { diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java index 7691586ac3e..db10c5a0926 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java @@ -76,7 +76,7 @@ public class TestZooKeeperScanPolicyObserver { .setTimeToLive(1); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); long now = EnvironmentEdgeManager.currentTime(); ZooKeeperWatcher zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(), "test", null); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java index 9e000879a81..75ef959486e 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java @@ -27,6 +27,8 @@ 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.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -224,7 +226,8 @@ public class IntegrationTestLazyCfLoading { long maxRuntime = conf.getLong(timeoutKey, DEFAULT_TIMEOUT_MINUTES); long serverCount = util.getHBaseClusterInterface().getClusterStatus().getServersSize(); long keysToWrite = serverCount * KEYS_TO_WRITE_PER_SERVER; - Table table = new HTable(conf, TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(TABLE_NAME); // Create multi-threaded writer and start it. We write multiple columns/CFs and verify // their integrity, therefore multi-put is necessary. @@ -288,5 +291,6 @@ public class IntegrationTestLazyCfLoading { Assert.assertEquals("There are write failures", 0, writer.getNumWriteFailures()); Assert.assertTrue("Writer is not done", isWriterDone); // Assert.fail("Boom!"); + connection.close(); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java index b1ea8e53126..7531f552370 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java @@ -20,40 +20,35 @@ package org.apache.hadoop.hbase.chaos.actions; import java.io.IOException; import java.util.Collection; +import java.util.List; import org.apache.commons.lang.math.RandomUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.client.Table; /** * Action that restarts an HRegionServer holding one of the regions of the table. */ public class RestartRsHoldingTableAction extends RestartActionBaseAction { - private final String tableName; + private final RegionLocator locator; - public RestartRsHoldingTableAction(long sleepTime, String tableName) { + public RestartRsHoldingTableAction(long sleepTime, RegionLocator locator) { super(sleepTime); - this.tableName = tableName; + this.locator = locator; } @Override public void perform() throws Exception { - HTable table = null; - try { - LOG.info("Performing action: Restart random RS holding table " + this.tableName); - Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration(); - table = new HTable(conf, TableName.valueOf(tableName)); - } catch (IOException e) { - LOG.debug("Error creating HTable used to get list of region locations.", e); - return; - } + LOG.info("Performing action: Restart random RS holding table " + this.locator.getName()); - Collection serverNames = table.getRegionLocations().values(); - ServerName[] nameArray = serverNames.toArray(new ServerName[serverNames.size()]); - - restartRs(nameArray[RandomUtils.nextInt(nameArray.length)], sleepTime); + List locations = locator.getAllRegionLocations(); + restartRs(locations.get(RandomUtils.nextInt(locations.size())).getServerName(), sleepTime); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java index 4b07f8ff8c2..3d72447f2bf 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java @@ -144,7 +144,7 @@ public class IntegrationTestImportTsv implements Configurable, Tool { setCaching(1000); }}; try { - table = new HTable(getConf(), tableName); + table = util.getConnection().getTable(tableName); Iterator resultsIt = table.getScanner(scan).iterator(); Iterator expectedIt = simple_expected.iterator(); while (resultsIt.hasNext() && expectedIt.hasNext()) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 63430a10ca4..feed8609ee3 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -189,7 +189,8 @@ public class IntegrationTestMTTR { // Set up the action that will restart a region server holding a region from our table // because this table should only have one region we should be good. - restartRSAction = new RestartRsHoldingTableAction(sleepTime, tableName.getNameAsString()); + restartRSAction = new RestartRsHoldingTableAction(sleepTime, + util.getConnection().getRegionLocator(tableName)); // Set up the action that will kill the region holding meta. restartMetaAction = new RestartRsHoldingMetaAction(sleepTime); @@ -478,7 +479,7 @@ public class IntegrationTestMTTR { public PutCallable(Future f) throws IOException { super(f); - this.table = new HTable(util.getConfiguration(), tableName); + this.table = util.getConnection().getTable(tableName); } @Override @@ -504,7 +505,7 @@ public class IntegrationTestMTTR { public ScanCallable(Future f) throws IOException { super(f); - this.table = new HTable(util.getConfiguration(), tableName); + this.table = util.getConnection().getTable(tableName); } @Override @@ -545,7 +546,7 @@ public class IntegrationTestMTTR { protected boolean doAction() throws Exception { Admin admin = null; try { - admin = new HBaseAdmin(util.getConfiguration()); + admin = util.getHBaseAdmin(); ClusterStatus status = admin.getClusterStatus(); return status != null; } finally { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index c7dae788319..931fba42b23 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -18,18 +18,7 @@ package org.apache.hadoop.hbase.test; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Random; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; - +import com.google.common.collect.Sets; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; @@ -48,26 +37,27 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; -import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; 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.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapper; import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; @@ -97,7 +87,17 @@ import org.apache.hadoop.util.ToolRunner; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Sets; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; /** * This is an integration test borrowed from goraci, written by Keith Turner, @@ -340,7 +340,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { byte[] id; long count = 0; int i; - HTable table; + Table table; + Connection connection; long numNodes; long wrap; int width; @@ -348,8 +349,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { @Override protected void setup(Context context) throws IOException, InterruptedException { id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID()); - Configuration conf = context.getConfiguration(); - instantiateHTable(conf); + this.connection = ConnectionFactory.createConnection(context.getConfiguration()); + instantiateHTable(); this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT); current = new byte[this.width][]; int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT); @@ -361,8 +362,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { } } - protected void instantiateHTable(Configuration conf) throws IOException { - table = new HTable(conf, getTableName(conf)); + protected void instantiateHTable() throws IOException { + table = connection.getTable(getTableName(connection.getConfiguration())); table.setAutoFlushTo(false); table.setWriteBufferSize(4 * 1024 * 1024); } @@ -370,6 +371,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { @Override protected void cleanup(Context context) throws IOException ,InterruptedException { table.close(); + connection.close(); } @Override @@ -876,7 +878,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { System.exit(-1); } - Table table = new HTable(getConf(), getTableName(getConf())); + Connection connection = ConnectionFactory.createConnection(getConf()); + Table table = connection.getTable(getTableName(getConf())); Scan scan = new Scan(); scan.setBatch(10000); @@ -906,6 +909,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { } scanner.close(); table.close(); + connection.close(); return 0; } @@ -926,9 +930,10 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { org.apache.hadoop.hbase.client.Delete delete = new org.apache.hadoop.hbase.client.Delete(val); - Table table = new HTable(getConf(), getTableName(getConf())); - table.delete(delete); - table.close(); + try (Connection connection = ConnectionFactory.createConnection(getConf()); + Table table = connection.getTable(getTableName(getConf()))) { + table.delete(delete); + } System.out.println("Delete successful"); return 0; @@ -970,7 +975,8 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null; int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1; - Table table = new HTable(getConf(), getTableName(getConf())); + Connection connection = ConnectionFactory.createConnection(getConf()); + Table table = connection.getTable(getTableName(getConf())); long numQueries = 0; // If isSpecificStart is set, only walk one list from that particular node. // Note that in case of circular (or P-shaped) list it will walk forever, as is @@ -1003,6 +1009,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { } table.close(); + connection.close(); return 0; } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java index dc517a5c558..50c638a496f 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java @@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory; 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.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; @@ -182,9 +184,9 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB } @Override - protected void instantiateHTable(Configuration conf) throws IOException { + protected void instantiateHTable() throws IOException { for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { - HTable table = new HTable(conf, getTableName(i)); + Table table = connection.getTable(getTableName(i)); table.setAutoFlushTo(true); //table.setWriteBufferSize(4 * 1024 * 1024); this.tables[i] = table; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index 60f20a5b0a1..6e10ba9e11f 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -40,6 +40,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.IntegrationTestBase; import org.apache.hadoop.hbase.IntegrationTestingUtility; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -164,13 +167,14 @@ public void cleanUpCluster() throws Exception { extends Mapper { protected long recordsToWrite; - protected HTable table; + protected Connection connection; + protected Table table; protected Configuration conf; protected int numBackReferencesPerRow; + protected String shortTaskId; protected Random rand = new Random(); - protected Counter rowsWritten, refsWritten; @Override @@ -179,7 +183,8 @@ public void cleanUpCluster() throws Exception { recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT); String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT); numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT); - table = new HTable(conf, TableName.valueOf(tableName)); + this.connection = ConnectionFactory.createConnection(conf); + table = connection.getTable(TableName.valueOf(tableName)); table.setWriteBufferSize(4*1024*1024); table.setAutoFlushTo(false); @@ -198,6 +203,7 @@ public void cleanUpCluster() throws Exception { public void cleanup(Context context) throws IOException { table.flushCommits(); table.close(); + connection.close(); } @Override diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java index b1cf57e3df4..1f313c348ae 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java @@ -124,7 +124,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { ResultScanner rs = null; try { innerScope = Trace.startSpan("Scan", Sampler.ALWAYS); - Table ht = new HTable(util.getConfiguration(), tableName); + Table ht = util.getConnection().getTable(tableName); Scan s = new Scan(); s.setStartRow(Bytes.toBytes(rowKeyQueue.take())); s.setBatch(7); @@ -174,7 +174,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { Table ht = null; try { - ht = new HTable(util.getConfiguration(), tableName); + ht = util.getConnection().getTable(tableName); } catch (IOException e) { e.printStackTrace(); } @@ -234,7 +234,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { private LinkedBlockingQueue insertData() throws IOException, InterruptedException { LinkedBlockingQueue rowKeys = new LinkedBlockingQueue(25000); - HTable ht = new HTable(util.getConfiguration(), this.tableName); + Table ht = util.getConnection().getTable(this.tableName); byte[] value = new byte[300]; for (int x = 0; x < 5000; x++) { TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java index 66483d72453..e4a322a3d75 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java @@ -101,7 +101,7 @@ public class TestGzipFilter { Response response = client.put(path, headers, value_1_gzip); assertEquals(response.getCode(), 200); - Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = TEST_UTIL.getConnection().getTable(TABLE); Get get = new Get(Bytes.toBytes(ROW_1)); get.addColumn(Bytes.toBytes(CFA), Bytes.toBytes("1")); Result result = table.get(get); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java index 7f0b1f56f9c..5bd8fc842b8 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java @@ -137,7 +137,7 @@ public class TestScannersWithFilters { htd.addFamily(new HColumnDescriptor(FAMILIES[0])); htd.addFamily(new HColumnDescriptor(FAMILIES[1])); admin.createTable(htd); - Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = TEST_UTIL.getConnection().getTable(TABLE); // Insert first half for(byte [] ROW : ROWS_ONE) { Put p = new Put(ROW); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java index 57459544bfc..41c036df6be 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java @@ -17,23 +17,6 @@ */ package org.apache.hadoop.hbase.rest; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.StringWriter; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Random; - -import javax.xml.bind.JAXBContext; -import javax.xml.bind.JAXBException; -import javax.xml.bind.Marshaller; -import javax.xml.bind.Unmarshaller; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; @@ -42,7 +25,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; @@ -69,6 +51,21 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; +import javax.xml.bind.Unmarshaller; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.StringWriter; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + @Category({RestTests.class, MediumTests.class}) public class TestScannersWithLabels { private static final TableName TABLE = TableName.valueOf("TestScannersWithLabels"); @@ -104,7 +101,7 @@ public class TestScannersWithLabels { + TOPSECRET)); puts.add(put); } - try (Table table = new HTable(TEST_UTIL.getConfiguration(), tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.put(puts); } return puts.size(); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java index 77e89cdd266..7cbb2a290b8 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableResource.java @@ -87,7 +87,7 @@ public class TestTableResource { HTableDescriptor htd = new HTableDescriptor(TABLE); htd.addFamily(new HColumnDescriptor(COLUMN_FAMILY)); admin.createTable(htd); - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLE); byte[] k = new byte[3]; byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(COLUMN)); for (byte b1 = 'a'; b1 < 'z'; b1++) { diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java index 951699568af..eb1fc98ddd6 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java @@ -101,7 +101,7 @@ public class TestRemoteTable { admin.createTable(htd); Table table = null; try { - table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + table = TEST_UTIL.getConnection().getTable(TABLE); Put put = new Put(ROW_1); put.add(COLUMN_1, QUALIFIER_1, TS_2, VALUE_1); table.put(put); @@ -135,7 +135,7 @@ public class TestRemoteTable { public void testGetTableDescriptor() throws IOException { Table table = null; try { - table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + table = TEST_UTIL.getConnection().getTable(TABLE); HTableDescriptor local = table.getTableDescriptor(); assertEquals(remoteTable.getTableDescriptor(), local); } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index f1fd7d2f30b..eacba6f040b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -30,7 +30,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; @@ -461,7 +462,8 @@ public class LocalHBaseCluster { Configuration conf = HBaseConfiguration.create(); LocalHBaseCluster cluster = new LocalHBaseCluster(conf); cluster.startup(); - Admin admin = new HBaseAdmin(conf); + Connection connection = ConnectionFactory.createConnection(conf); + Admin admin = connection.getAdmin(); try { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(cluster.getClass().getName())); @@ -469,6 +471,7 @@ public class LocalHBaseCluster { } finally { admin.close(); } + connection.close(); cluster.shutdown(); } } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 312a3ae8c05..eeb941ab268 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.HTableWrapper; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CoprocessorClassLoader; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.VersionInfo; @@ -325,7 +324,7 @@ public abstract class CoprocessorHost { final ClassLoader systemClassLoader = this.getClass().getClassLoader(); for (E env : coprocessors) { ClassLoader cl = env.getInstance().getClass().getClassLoader(); - if (cl != systemClassLoader ){ + if (cl != systemClassLoader){ //do not include system classloader externalClassLoaders.add(cl); } @@ -434,7 +433,7 @@ public abstract class CoprocessorHost { } catch (IOException e) { // nothing can be done here LOG.warn("Failed to close " + - Bytes.toStringBinary(table.getTableName()), e); + table.getName(), e); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java index e1220fbc18d..368510fd78a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java @@ -25,8 +25,9 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapred.FileInputFormat; @@ -58,8 +59,8 @@ public class TableInputFormat extends TableInputFormatBase implements } setInputColumns(m_cols); try { - setHTable( - new HTable(HBaseConfiguration.create(job), TableName.valueOf(tableNames[0].getName()))); + Connection connection = ConnectionFactory.createConnection(job); + setHTable((HTable) connection.getTable(TableName.valueOf(tableNames[0].getName()))); } catch (Exception e) { LOG.error(StringUtils.stringifyException(e)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java index fbfd984354c..2a50efc4aac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.mapred.InputFormat; @@ -158,7 +159,7 @@ implements InputFormat { /** * Allows subclasses to get the {@link HTable}. */ - protected HTable getHTable() { + protected Table getHTable() { return this.table; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java index e88d6dff7bf..7584bc25586 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CopyTable.java @@ -34,10 +34,10 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +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.Scan; -import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.util.GenericOptionsParser; @@ -156,8 +156,9 @@ public class CopyTable extends Configured implements Tool { System.out.println("HFiles will be stored at " + this.bulkloadDir); HFileOutputFormat2.setOutputPath(job, bulkloadDir); try (Connection conn = ConnectionFactory.createConnection(getConf()); - Table htable = conn.getTable(TableName.valueOf(dstTableName))) { - HFileOutputFormat2.configureIncrementalLoadMap(job, htable); + Admin admin = conn.getAdmin()) { + HFileOutputFormat2.configureIncrementalLoadMap(job, + admin.getTableDescriptor((TableName.valueOf(dstTableName)))); } } else { TableMapReduceUtil.initTableMapperJob(tableName, scan, @@ -192,7 +193,8 @@ public class CopyTable extends Configured implements Tool { System.err.println(" versions number of cell versions to copy"); System.err.println(" new.name new table's name"); System.err.println(" peer.adr Address of the peer cluster given in the format"); - System.err.println(" hbase.zookeeer.quorum:hbase.zookeeper.client.port:zookeeper.znode.parent"); + System.err.println(" hbase.zookeeper.quorum:hbase.zookeeper.client" + + ".port:zookeeper.znode.parent"); System.err.println(" families comma-separated list of families to copy"); System.err.println(" To copy from cf1 to cf2, give sourceCfName:destCfName. "); System.err.println(" To keep the same name, just give \"cfName\""); @@ -298,7 +300,7 @@ public class CopyTable extends Configured implements Tool { if (i == args.length-1) { tableName = cmd; } else { - printUsage("Invalid argument '" + cmd + "'" ); + printUsage("Invalid argument '" + cmd + "'"); return false; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index 9ccaf95dc3a..f69f21f4b45 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -449,7 +449,8 @@ public class HFileOutputFormat2 LOG.info("Incremental table " + regionLocator.getName() + " output configured."); } - public static void configureIncrementalLoadMap(Job job, Table table) throws IOException { + public static void configureIncrementalLoadMap(Job job, HTableDescriptor tableDescriptor) throws + IOException { Configuration conf = job.getConfiguration(); job.setOutputKeyClass(ImmutableBytesWritable.class); @@ -457,15 +458,14 @@ public class HFileOutputFormat2 job.setOutputFormatClass(HFileOutputFormat2.class); // Set compression algorithms based on column families - configureCompression(conf, table.getTableDescriptor()); - configureBloomType(table.getTableDescriptor(), conf); - configureBlockSize(table.getTableDescriptor(), conf); - HTableDescriptor tableDescriptor = table.getTableDescriptor(); + configureCompression(conf, tableDescriptor); + configureBloomType(tableDescriptor, conf); + configureBlockSize(tableDescriptor, conf); configureDataBlockEncoding(tableDescriptor, conf); TableMapReduceUtil.addDependencyJars(job); TableMapReduceUtil.initCredentials(job); - LOG.info("Incremental table " + table.getName() + " output configured."); + LOG.info("Incremental table " + tableDescriptor.getTableName() + " output configured."); } /** @@ -483,8 +483,7 @@ public class HFileOutputFormat2 Map compressionMap = new TreeMap(Bytes.BYTES_COMPARATOR); for (Map.Entry e : stringMap.entrySet()) { - Algorithm algorithm = AbstractHFileWriter.compressionByName - (e.getValue()); + Algorithm algorithm = AbstractHFileWriter.compressionByName(e.getValue()); compressionMap.put(e.getKey(), algorithm); } return compressionMap; @@ -602,7 +601,7 @@ public class HFileOutputFormat2 * Serialize column family to compression algorithm map to configuration. * Invoked while configuring the MR job for incremental load. * - * @param table to read the properties from + * @param tableDescriptor to read the properties from * @param conf to persist serialized values into * @throws IOException * on failure to read column family descriptors @@ -705,7 +704,7 @@ public class HFileOutputFormat2 * Serialize column family to data block encoding map to configuration. * Invoked while configuring the MR job for incremental load. * - * @param table to read the properties from + * @param tableDescriptor to read the properties from * @param conf to persist serialized values into * @throws IOException * on failure to read column family descriptors diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java index bd445185154..9c5b5afd9ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +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.Delete; @@ -512,18 +513,23 @@ public class Import extends Configured implements Tool { public static void flushRegionsIfNecessary(Configuration conf) throws IOException, InterruptedException { String tableName = conf.get(TABLE_NAME); - HBaseAdmin hAdmin = null; + Admin hAdmin = null; + Connection connection = null; String durability = conf.get(WAL_DURABILITY); // Need to flush if the data is written to hbase and skip wal is enabled. if (conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)) { try { - hAdmin = new HBaseAdmin(conf); - hAdmin.flush(tableName); + connection = ConnectionFactory.createConnection(conf); + hAdmin = connection.getAdmin(); + hAdmin.flush(TableName.valueOf(tableName)); } finally { if (hAdmin != null) { hAdmin.close(); } + if (connection != null) { + connection.close(); + } } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java index 90f2f0e51df..d4394ebd2e9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java @@ -477,7 +477,7 @@ public class ImportTsv extends Configured implements Tool { job.setInputFormatClass(TextInputFormat.class); job.setMapperClass(mapperClass); String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY); - String columns[] = conf.getStrings(COLUMNS_CONF_KEY); + String[] columns = conf.getStrings(COLUMNS_CONF_KEY); if(StringUtils.isNotEmpty(conf.get(CREDENTIALS_LOCATION))) { String fileLoc = conf.get(CREDENTIALS_LOCATION); Credentials cred = Credentials.readTokenStorageFile(new File(fileLoc), conf); @@ -662,7 +662,7 @@ public class ImportTsv extends Configured implements Tool { // TODO: validation for TsvImporterMapper, not this tool. Move elsewhere. if (null == getConf().get(MAPPER_CONF_KEY)) { // Make sure columns are specified - String columns[] = getConf().getStrings(COLUMNS_CONF_KEY); + String[] columns = getConf().getStrings(COLUMNS_CONF_KEY); if (columns == null) { usage("No columns specified. Please specify with -D" + COLUMNS_CONF_KEY+"=..."); 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 51a5458d87d..7866c100558 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 @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; 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.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HTable; @@ -932,7 +933,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool { } Path hfofDir = new Path(dirPath); - HTable table = new HTable(getConf(), tableName); + Connection connection = ConnectionFactory.createConnection(getConf()); + HTable table = (HTable) connection.getTable(tableName); doBulkLoad(hfofDir, table); return 0; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java index 20cf50a7893..0c8e76f22f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java @@ -29,11 +29,13 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.JobContext; @@ -73,7 +75,8 @@ public class MultiTableOutputFormat extends OutputFormat { private static final Log LOG = LogFactory.getLog(MultiTableRecordWriter.class); - Map tables; + Connection connection; + Map tables; Configuration conf; boolean useWriteAheadLogging; @@ -85,10 +88,10 @@ public class MultiTableOutputFormat extends OutputFormatfalse) to improve performance when bulk loading data. */ public MultiTableRecordWriter(Configuration conf, - boolean useWriteAheadLogging) { + boolean useWriteAheadLogging) throws IOException { LOG.debug("Created new MultiTableRecordReader with WAL " + (useWriteAheadLogging ? "on" : "off")); - this.tables = new HashMap(); + this.tables = new HashMap(); this.conf = conf; this.useWriteAheadLogging = useWriteAheadLogging; } @@ -100,10 +103,14 @@ public class MultiTableOutputFormat extends OutputFormat { + + public static enum Counters { GOODROWS, BADROWS, ONLY_IN_SOURCE_TABLE_ROWS, ONLY_IN_PEER_TABLE_ROWS, CONTENT_DIFFERENT_ROWS} + private Connection connection; + private Table replicatedTable; private ResultScanner replicatedScanner; private Result currentCompareRowInPeerTable; @@ -129,8 +137,8 @@ public class VerifyReplication extends Configured implements Tool { ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey); TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName")); - // TODO: THis HTable doesn't get closed. Fix! - Table replicatedTable = new HTable(peerConf, tableName); + connection = ConnectionFactory.createConnection(peerConf); + replicatedTable = connection.getTable(tableName); scan.setStartRow(value.getRow()); scan.setStopRow(tableSplit.getEndRow()); replicatedScanner = replicatedTable.getScanner(scan); @@ -191,6 +199,20 @@ public class VerifyReplication extends Configured implements Tool { replicatedScanner = null; } } + if(replicatedTable != null){ + try{ + replicatedTable.close(); + } catch (Exception e) { + LOG.error("fail to close table in cleanup", e); + } + } + if(connection != null){ + try { + connection.close(); + } catch (Exception e) { + LOG.error("fail to close connection in cleanup", e); + } + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java index c577abfab70..78c7a065c56 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java @@ -37,12 +37,13 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; 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.Delete; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnectable; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Table; @@ -245,7 +246,8 @@ class HMerge { throws IOException { super(conf, fs, tableName); this.tableName = tableName; - this.table = new HTable(conf, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + this.table = connection.getTable(TableName.META_TABLE_NAME); this.metaScanner = table.getScanner(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); this.latestRegion = null; diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index 1f1871c6bda..c9878bf8ee0 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -24,7 +24,6 @@ import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.hbase.client.HTable" import="org.apache.hadoop.hbase.client.Admin" - import="org.apache.hadoop.hbase.client.HConnectionManager" import="org.apache.hadoop.hbase.HRegionInfo" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.ServerLoad" @@ -33,17 +32,16 @@ import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.util.FSUtils" - import="org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest" import="org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState" import="org.apache.hadoop.hbase.TableName" - import="org.apache.hadoop.hbase.client.RegionReplicaUtil" import="org.apache.hadoop.hbase.HBaseConfiguration" %> <% HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER); Configuration conf = master.getConfiguration(); + MetaTableLocator metaTableLocator = new MetaTableLocator(); String fqtn = request.getParameter("name"); - HTable table = new HTable(conf, fqtn); + HTable table = (HTable) master.getConnection().getTable(fqtn); String tableHeader; boolean withReplica = false; if (table.getTableDescriptor().getRegionReplication() > 1) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 6b4394f6409..40b136462d7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -17,34 +17,6 @@ */ package org.apache.hadoop.hbase; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.File; -import java.io.IOException; -import java.io.OutputStream; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; -import java.net.InetAddress; -import java.net.ServerSocket; -import java.net.Socket; -import java.net.UnknownHostException; -import java.security.MessageDigest; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.NavigableSet; -import java.util.Random; -import java.util.Set; -import java.util.TreeSet; -import java.util.UUID; -import java.util.concurrent.TimeUnit; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.impl.Jdk14Logger; @@ -90,7 +62,6 @@ import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.tool.Canary; import org.apache.hadoop.hbase.util.Bytes; @@ -103,6 +74,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.RegionSplitter; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.zookeeper.EmptyWatcher; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKConfig; @@ -118,6 +90,34 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper.States; +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.UnknownHostException; +import java.security.MessageDigest; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Random; +import java.util.Set; +import java.util.TreeSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Facility for testing HBase. Replacement for * old HBaseTestCase and HBaseClusterTestCase functionality. @@ -976,7 +976,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { this.hbaseCluster = new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass); // Don't leave here till we've done a successful scan of the hbase:meta - Table t = new HTable(c, TableName.META_TABLE_NAME); + Table t = getConnection().getTable(TableName.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { continue; @@ -996,6 +996,10 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @throws IOException */ public void restartHBaseCluster(int servers) throws IOException, InterruptedException { + if(connection != null){ + connection.close(); + connection = null; + } this.hbaseCluster = new MiniHBaseCluster(this.conf, servers); // Don't leave here till we've done a successful scan of the hbase:meta Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); @@ -1276,7 +1280,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc, startKey, endKey, numRegions); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(getConfiguration(), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1315,7 +1319,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(htd, splitRows); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(htd.getTableName()); - return new HTable(getConfiguration(), htd.getTableName()); + return (HTable) getConnection().getTable(htd.getTableName()); } /** @@ -1353,7 +1357,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { desc.addFamily(hcd); } getHBaseAdmin().createTable(desc); - return new HTable(c, desc.getTableName()); + return (HTable) getConnection().getTable(desc.getTableName()); } /** @@ -1377,7 +1381,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(c, tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1399,7 +1403,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { desc.addFamily(hcd); } getHBaseAdmin().createTable(desc); - return new HTable(c, desc.getTableName()); + return (HTable) getConnection().getTable(desc.getTableName()); } /** @@ -1461,7 +1465,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1498,7 +1502,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1537,7 +1541,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(new Configuration(getConfiguration()), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1569,7 +1573,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc, splitRows); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(tableName); - return new HTable(getConfiguration(), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -1590,7 +1594,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { getHBaseAdmin().createTable(desc, splitRows); // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are assigned waitUntilAllRegionsAssigned(desc.getTableName()); - return new HTable(getConfiguration(), desc.getTableName()); + return (HTable) getConnection().getTable(desc.getTableName()); } /** @@ -1809,7 +1813,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @throws IOException */ public HTable deleteTableData(TableName tableName) throws IOException { - HTable table = new HTable(getConfiguration(), tableName); + HTable table = (HTable) getConnection().getTable(tableName); Scan scan = new Scan(); ResultScanner resScan = table.getScanner(scan); for(Result res : resScan) { @@ -1831,7 +1835,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public HTable truncateTable(final TableName tableName, final boolean preserveRegions) throws IOException { Admin admin = getHBaseAdmin(); admin.truncateTable(tableName, preserveRegions); - return new HTable(getConfiguration(), tableName); + return (HTable) getConnection().getTable(tableName); } /** @@ -2271,7 +2275,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { public List createMultiRegionsInMeta(final Configuration conf, final HTableDescriptor htd, byte [][] startKeys) throws IOException { - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = (HTable) getConnection().getTable(TableName.META_TABLE_NAME); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList(startKeys.length); // add custom ones @@ -2294,7 +2298,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public List getMetaTableRows() throws IOException { // TODO: Redo using MetaTableAccessor class - Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); + Table t = (HTable) getConnection().getTable(TableName.META_TABLE_NAME); List rows = new ArrayList(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2314,7 +2318,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public List getMetaTableRows(TableName tableName) throws IOException { // TODO: Redo using MetaTableAccessor. - Table t = new HTable(new Configuration(this.conf), TableName.META_TABLE_NAME); + Table t = getConnection().getTable(TableName.META_TABLE_NAME); List rows = new ArrayList(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2631,7 +2635,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { monitor.close(); if (checkStatus) { - new HTable(new Configuration(conf), TableName.META_TABLE_NAME).close(); + getConnection().getTable(TableName.META_TABLE_NAME).close(); } } @@ -3140,7 +3144,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { */ public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { - final Table meta = new HTable(getConfiguration(), TableName.META_TABLE_NAME); + final Table meta = getConnection().getTable(TableName.META_TABLE_NAME); try { waitFor(timeout, 200, true, new Predicate() { @Override @@ -3548,9 +3552,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return totalNumberOfRegions; } - public static int getMetaRSPort(Configuration conf) throws IOException { - try (Connection c = ConnectionFactory.createConnection(); - RegionLocator locator = c.getRegionLocator(TableName.META_TABLE_NAME)) { + public static int getMetaRSPort(Connection connection) throws IOException { + try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) { return locator.getRegionLocation(Bytes.toBytes("")).getPort(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 7524d5c993c..52424b30d81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -1020,7 +1020,7 @@ public class PerformanceEvaluation extends Configured implements Tool { if (!opts.oneCon) { this.connection = ConnectionFactory.createConnection(conf); } - this.table = new HTable(TableName.valueOf(opts.tableName), connection); + this.table = connection.getTable(TableName.valueOf(opts.tableName)); this.table.setAutoFlushTo(opts.autoFlush); latency = YammerHistogramUtils.newHistogram(new UniformSample(1024 * 500)); valueSize = YammerHistogramUtils.newHistogram(new UniformSample(1024 * 500)); @@ -1568,11 +1568,14 @@ public class PerformanceEvaluation extends Configured implements Tool { // the TestOptions introspection for us and dump the output in a readable format. LOG.info(cmd.getSimpleName() + " test run options=" + MAPPER.writeValueAsString(opts)); Admin admin = null; + Connection connection = null; try { - admin = new HBaseAdmin(getConf()); + connection = ConnectionFactory.createConnection(getConf()); + admin = connection.getAdmin(); checkTable(admin, opts); } finally { if (admin != null) admin.close(); + if (connection != null) connection.close(); } if (opts.nomapred) { doLocalClients(opts, getConf()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java index 6b8e63d668e..0c331b76147 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ScanPerformanceEvaluation.java @@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -149,7 +151,8 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool { Stopwatch scanTimer = new Stopwatch(); tableOpenTimer.start(); - Table table = new HTable(getConf(), TableName.valueOf(tablename)); + Connection connection = ConnectionFactory.createConnection(getConf()); + Table table = connection.getTable(TableName.valueOf(tablename)); tableOpenTimer.stop(); Scan scan = getScan(); @@ -172,6 +175,7 @@ public class ScanPerformanceEvaluation extends AbstractHBaseTool { scanTimer.stop(); scanner.close(); table.close(); + connection.close(); ScanMetrics metrics = ProtobufUtil.toScanMetrics(scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA)); long totalBytes = metrics.countOfBytesInResults.get(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java index 3068fbfcb54..5659f6b1c2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestAcidGuarantees.java @@ -28,6 +28,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread; import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; @@ -113,7 +115,8 @@ public class TestAcidGuarantees implements Tool { super(ctx); this.targetRows = targetRows; this.targetFamilies = targetFamilies; - table = new HTable(ctx.getConf(), TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(ctx.getConf()); + table = connection.getTable(TABLE_NAME); } public void doAnAction() throws Exception { // Pick a random row to write into @@ -148,7 +151,8 @@ public class TestAcidGuarantees implements Tool { super(ctx); this.targetRow = targetRow; this.targetFamilies = targetFamilies; - table = new HTable(ctx.getConf(), TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(ctx.getConf()); + table = connection.getTable(TABLE_NAME); } public void doAnAction() throws Exception { @@ -205,7 +209,8 @@ public class TestAcidGuarantees implements Tool { byte targetFamilies[][]) throws IOException { super(ctx); this.targetFamilies = targetFamilies; - table = new HTable(ctx.getConf(), TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(ctx.getConf()); + table = connection.getTable(TABLE_NAME); } public void doAnAction() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index 62b00d8d24e..a72b151349a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -67,7 +67,7 @@ public class TestInfoServers { @Test public void testInfoServersRedirect() throws Exception { // give the cluster time to start up - new HTable(UTIL.getConfiguration(), TableName.META_TABLE_NAME).close(); + UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertContainsContent(new URL("http://localhost:" + port + "/index.html"), "master-status"); @@ -87,7 +87,7 @@ public class TestInfoServers { @Test public void testInfoServersStatusPages() throws Exception { // give the cluster time to start up - new HTable(UTIL.getConfiguration(), TableName.META_TABLE_NAME).close(); + UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertContainsContent(new URL("http://localhost:" + port + "/master-status"), "meta"); @@ -102,7 +102,7 @@ public class TestInfoServers { TableName tableName = TableName.valueOf("testMasterServerReadOnly"); byte[] cf = Bytes.toBytes("d"); UTIL.createTable(tableName, cf); - new HTable(UTIL.getConfiguration(), tableName).close(); + UTIL.getConnection().getTable(tableName).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertDoesNotContainContent( new URL("http://localhost:" + port + "/table.jsp?name=" + tableName + "&action=split&key="), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java index 3491d723f0b..711d5925567 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -77,12 +78,7 @@ public class TestMultiVersions { @Before public void before() throws MasterNotRunningException, ZooKeeperConnectionException, IOException { - this.admin = new HBaseAdmin(UTIL.getConfiguration()); - } - - @After - public void after() throws IOException { - this.admin.close(); + this.admin = UTIL.getHBaseAdmin(); } /** @@ -101,7 +97,7 @@ public class TestMultiVersions { hcd.setMaxVersions(3); desc.addFamily(hcd); this.admin.createTable(desc); - Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table table = UTIL.getConnection().getTable(desc.getTableName()); // TODO: Remove these deprecated classes or pull them in here if this is // only test using them. Incommon incommon = new HTableIncommon(table); @@ -144,16 +140,15 @@ public class TestMultiVersions { this.admin.createTable(desc); Put put = new Put(row, timestamp1); put.add(contents, contents, value1); - Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table table = UTIL.getConnection().getTable(desc.getTableName()); table.put(put); // Shut down and restart the HBase cluster table.close(); UTIL.shutdownMiniHBaseCluster(); LOG.debug("HBase cluster shut down -- restarting"); UTIL.startMiniHBaseCluster(1, NUM_SLAVES); - // Make a new connection. Use new Configuration instance because old one - // is tied to an HConnection that has since gone stale. - table = new HTable(new Configuration(UTIL.getConfiguration()), desc.getTableName()); + // Make a new connection. + table = UTIL.getConnection().getTable(desc.getTableName()); // Overwrite previous value put = new Put(row, timestamp2); put.add(contents, contents, value2); @@ -207,23 +202,25 @@ public class TestMultiVersions { final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")}; final long [] timestamp = new long[] {100L, 1000L}; this.admin.createTable(desc, splitRows); - HTable table = new HTable(UTIL.getConfiguration(), tableName); + Table table = UTIL.getConnection().getTable(tableName); // Assert we got the region layout wanted. - NavigableMap locations = table.getRegionLocations(); - assertEquals(2, locations.size()); - int index = 0; - for (HRegionInfo hri: locations.keySet()) { - if (index == 0) { - assertTrue(Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey())); - assertTrue(Bytes.equals(hri.getEndKey(), splitRows[0])); - } else if (index == 1) { - assertTrue(Bytes.equals(splitRows[0], hri.getStartKey())); - assertTrue(Bytes.equals(hri.getEndKey(), HConstants.EMPTY_END_ROW)); + Pair keys = UTIL.getConnection() + .getRegionLocator(tableName).getStartEndKeys(); + assertEquals(2, keys.getFirst().length); + byte[][] startKeys = keys.getFirst(); + byte[][] endKeys = keys.getSecond(); + + for (int i = 0; i < startKeys.length; i++) { + if (i == 0) { + assertTrue(Bytes.equals(HConstants.EMPTY_START_ROW, startKeys[i])); + assertTrue(Bytes.equals(endKeys[i], splitRows[0])); + } else if (i == 1) { + assertTrue(Bytes.equals(splitRows[0], startKeys[i])); + assertTrue(Bytes.equals(endKeys[i], HConstants.EMPTY_END_ROW)); } - index++; } // Insert data - for (int i = 0; i < locations.size(); i++) { + for (int i = 0; i < startKeys.length; i++) { for (int j = 0; j < timestamp.length; j++) { Put put = new Put(rows[i], timestamp[j]); put.add(HConstants.CATALOG_FAMILY, null, timestamp[j], diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index baa43fab2dc..f47a8e04f40 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -252,7 +252,7 @@ public class TestNamespace { } //sanity check try to write and read from table - Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); Put p = new Put(Bytes.toBytes("row1")); p.add(Bytes.toBytes("my_cf"),Bytes.toBytes("my_col"),Bytes.toBytes("value1")); table.put(p); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index 1944b611190..30ad325da5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -35,11 +35,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; 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.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -127,7 +127,7 @@ public class TestZooKeeper { } } - private ZooKeeperWatcher getZooKeeperWatcher(HConnection c) + private ZooKeeperWatcher getZooKeeperWatcher(Connection c) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { Method getterZK = c.getClass().getDeclaredMethod("getKeepAliveZooKeeperWatcher"); getterZK.setAccessible(true); @@ -148,7 +148,7 @@ public class TestZooKeeper { // We don't want to share the connection as we will check its state c.set(HConstants.HBASE_CLIENT_INSTANCE_ID, "1111"); - HConnection connection = HConnectionManager.getConnection(c); + Connection connection = ConnectionFactory.createConnection(c); ZooKeeperWatcher connectionZK = getZooKeeperWatcher(connection); LOG.info("ZooKeeperWatcher= 0x"+ Integer.toHexString( @@ -253,15 +253,14 @@ public class TestZooKeeper { HColumnDescriptor family = new HColumnDescriptor("fam"); desc.addFamily(family); LOG.info("Creating table " + tableName); - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); try { admin.createTable(desc); } finally { admin.close(); } - Table table = - new HTable(new Configuration(TEST_UTIL.getConfiguration()), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); Put put = new Put(Bytes.toBytes("testrow")); put.add(Bytes.toBytes("fam"), Bytes.toBytes("col"), Bytes.toBytes("testdata")); @@ -273,11 +272,11 @@ public class TestZooKeeper { @Test public void testMultipleZK() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { - Table localMeta = - new HTable(new Configuration(TEST_UTIL.getConfiguration()), TableName.META_TABLE_NAME); + Table localMeta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1"); - Table ipMeta = new HTable(otherConf, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(otherConf); + Table ipMeta = connection.getTable(TableName.META_TABLE_NAME); // dummy, just to open the connection final byte [] row = new byte [] {'r'}; @@ -294,6 +293,7 @@ public class TestZooKeeper { localMeta.close(); ipMeta.close(); + connection.close(); } /** @@ -494,7 +494,7 @@ public class TestZooKeeper { ZooKeeperWatcher zkw = m.getZooKeeper(); int expectedNumOfListeners = zkw.getNumberOfListeners(); // now the cluster is up. So assign some regions. - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); try { byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d"), Bytes.toBytes("e"), Bytes.toBytes("f"), @@ -531,7 +531,7 @@ public class TestZooKeeper { cluster.startRegionServer(); HMaster m = cluster.getMaster(); // now the cluster is up. So assign some regions. - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); Table table = null; try { byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("1"), Bytes.toBytes("2"), @@ -543,7 +543,7 @@ public class TestZooKeeper { htd.addFamily(hcd); admin.createTable(htd, SPLIT_KEYS); TEST_UTIL.waitUntilNoRegionsInTransition(60000); - table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()); + table = TEST_UTIL.getConnection().getTable(htd.getTableName()); Put p; int numberOfPuts; for (numberOfPuts = 0; numberOfPuts < 6; numberOfPuts++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index bc36fa9155e..9b55acd2d8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -367,7 +367,7 @@ public class TestAdmin1 { splitKeys[1] = Bytes.toBytes(8); // Create & Fill the table - HTable table = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY, splitKeys); + Table table = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY, splitKeys); try { TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10); assertEquals(10, TEST_UTIL.countRows(table)); @@ -379,7 +379,7 @@ public class TestAdmin1 { // Truncate & Verify this.admin.disableTable(tableName); this.admin.truncateTable(tableName, preserveSplits); - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); try { assertEquals(0, TEST_UTIL.countRows(table)); } finally { @@ -402,7 +402,7 @@ public class TestAdmin1 { htd.addFamily(fam2); htd.addFamily(fam3); this.admin.createTable(htd); - Table table = new HTable(TEST_UTIL.getConfiguration(), htd.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); HTableDescriptor confirmedHtd = table.getTableDescriptor(); assertEquals(htd.compareTo(confirmedHtd), 0); table.close(); @@ -585,7 +585,7 @@ public class TestAdmin1 { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); Map regions = ht.getRegionLocations(); assertEquals("Table should have only 1 region", 1, regions.size()); ht.close(); @@ -594,7 +594,7 @@ public class TestAdmin1 { desc = new HTableDescriptor(TABLE_2); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, new byte[][]{new byte[]{42}}); - HTable ht2 = new HTable(TEST_UTIL.getConfiguration(), TABLE_2); + HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2); regions = ht2.getRegionLocations(); assertEquals("Table should have only 2 region", 2, regions.size()); ht2.close(); @@ -603,7 +603,7 @@ public class TestAdmin1 { desc = new HTableDescriptor(TABLE_3); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3); - HTable ht3 = new HTable(TEST_UTIL.getConfiguration(), TABLE_3); + HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3); regions = ht3.getRegionLocations(); assertEquals("Table should have only 3 region", 3, regions.size()); ht3.close(); @@ -622,7 +622,7 @@ public class TestAdmin1 { desc = new HTableDescriptor(TABLE_5); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16); - HTable ht5 = new HTable(TEST_UTIL.getConfiguration(), TABLE_5); + HTable ht5 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_5); regions = ht5.getRegionLocations(); assertEquals("Table should have 16 region", 16, regions.size()); ht5.close(); @@ -653,7 +653,7 @@ public class TestAdmin1 { boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys); assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); Map regions = ht.getRegionLocations(); assertEquals("Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), @@ -710,10 +710,10 @@ public class TestAdmin1 { desc = new HTableDescriptor(TABLE_2); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + admin = TEST_UTIL.getHBaseAdmin(); admin.createTable(desc, startKey, endKey, expectedRegions); - HTable ht2 = new HTable(TEST_UTIL.getConfiguration(), TABLE_2); + HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2); regions = ht2.getRegionLocations(); assertEquals("Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), @@ -766,11 +766,11 @@ public class TestAdmin1 { desc = new HTableDescriptor(TABLE_3); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + admin = TEST_UTIL.getHBaseAdmin(); admin.createTable(desc, startKey, endKey, expectedRegions); - HTable ht3 = new HTable(TEST_UTIL.getConfiguration(), TABLE_3); + HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3); regions = ht3.getRegionLocations(); assertEquals("Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), @@ -792,15 +792,13 @@ public class TestAdmin1 { TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4"); desc = new HTableDescriptor(TABLE_4); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); - Admin ladmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); try { - ladmin.createTable(desc, splitKeys); + admin.createTable(desc, splitKeys); assertTrue("Should not be able to create this table because of " + "duplicate split keys", false); } catch(IllegalArgumentException iae) { // Expected } - ladmin.close(); } @Test (timeout=300000) @@ -893,7 +891,7 @@ public class TestAdmin1 { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc, splitKeys); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); Map regions = ht.getRegionLocations(); assertEquals("Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(), expectedRegions, regions.size()); @@ -1104,7 +1102,7 @@ public class TestAdmin1 { Thread.sleep(10); } while (oldRegions.size() != 9); //3 regions * 3 replicas // write some data to the table - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName); List puts = new ArrayList(); byte[] qualifier = "c".getBytes(); Put put = new Put(new byte[]{(byte)'1'}); @@ -1227,7 +1225,7 @@ public class TestAdmin1 { } this.admin.disableTable(tableName); try { - new HTable(TEST_UTIL.getConfiguration(), tableName); + TEST_UTIL.getConnection().getTable(tableName); } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) { //expected } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 9daf6856f0b..bdf7de24670 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -176,7 +176,7 @@ public class TestAdmin2 { admin.createTable(htd1); admin.createTable(htd2); // Before fix, below would fail throwing a NoServerForRegionException. - new HTable(TEST_UTIL.getConfiguration(), htd2.getTableName()).close(); + TEST_UTIL.getConnection().getTable(htd2.getTableName()).close(); } /*** @@ -197,11 +197,10 @@ public class TestAdmin2 { // Use 80 bit numbers to make sure we aren't limited byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }; byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }; - Admin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin hbaseadmin = TEST_UTIL.getHBaseAdmin(); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); hbaseadmin.createTable(htd, startKey, endKey, expectedRegions); - hbaseadmin.close(); } finally { TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout); } @@ -299,7 +298,7 @@ public class TestAdmin2 { public void testTableNotFoundExceptionWithoutAnyTables() throws IOException { TableName tableName = TableName .valueOf("testTableNotFoundExceptionWithoutAnyTables"); - Table ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table ht = TEST_UTIL.getConnection().getTable(tableName); ht.get(new Get("e".getBytes())); } @@ -466,9 +465,7 @@ public class TestAdmin2 { } private HBaseAdmin createTable(byte[] TABLENAME) throws IOException { - - Configuration config = TEST_UTIL.getConfiguration(); - HBaseAdmin admin = new HBaseAdmin(config); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLENAME)); HColumnDescriptor hcd = new HColumnDescriptor("value"); @@ -608,14 +605,13 @@ public class TestAdmin2 { private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running - new HTable( - TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME).close(); + TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); // Create the test table and open it HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls @@ -716,7 +712,7 @@ public class TestAdmin2 { public void testGetRegion() throws Exception { // We use actual HBaseAdmin instance instead of going via Admin interface in // here because makes use of an internal HBA method (TODO: Fix.). - HBaseAdmin rawAdmin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin rawAdmin = TEST_UTIL.getHBaseAdmin(); final TableName tableName = TableName.valueOf("testGetRegion"); LOG.info("Started " + tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java index 6d58d039c3d..072098e1157 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientOperationInterrupt.java @@ -82,9 +82,8 @@ public class TestClientOperationInterrupt { } admin.deleteTable(tableName); } - util.createTable(tableName, new byte[][]{dummy, test}); + Table ht = util.createTable(tableName, new byte[][]{dummy, test}); - Table ht = new HTable(conf, tableName); Put p = new Put(row1); p.add(dummy, dummy, dummy); ht.put(p); @@ -106,7 +105,7 @@ public class TestClientOperationInterrupt { @Override public void run() { try { - Table ht = new HTable(conf, tableName); + Table ht = util.getConnection().getTable(tableName); Result r = ht.get(new Get(row1)); noEx.incrementAndGet(); } catch (IOException e) { @@ -155,7 +154,7 @@ public class TestClientOperationInterrupt { Thread.sleep(1); } - Table ht = new HTable(conf, tableName); + Table ht = util.getConnection().getTable(tableName); Result r = ht.get(new Get(row1)); Assert.assertFalse(r.isEmpty()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index c04c4f284df..d3986b2cdc2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -98,12 +98,11 @@ public class TestClientTimeouts { // Ensure the HBaseAdmin uses a new connection by changing Configuration. Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration()); conf.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1)); - HBaseAdmin admin = null; + Admin admin = null; + Connection connection = null; try { - admin = new HBaseAdmin(conf); - Connection connection = admin.getConnection(); - assertFalse(connection == lastConnection); - lastConnection = connection; + connection = ConnectionFactory.createConnection(conf); + admin = connection.getAdmin(); // run some admin commands HBaseAdmin.checkHBaseAvailable(conf); admin.setBalancerRunning(false, false); @@ -112,10 +111,15 @@ public class TestClientTimeouts { // a MasterNotRunningException. It's a bug if we get other exceptions. lastFailed = true; } finally { - admin.close(); - if (admin.getConnection().isClosed()) { - rpcClient = (RandomTimeoutRpcClient) RpcClientFactory - .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + if(admin != null) { + admin.close(); + if (admin.getConnection().isClosed()) { + rpcClient = (RandomTimeoutRpcClient) RpcClientFactory + .createClient(TEST_UTIL.getConfiguration(), TEST_UTIL.getClusterKey()); + } + } + if(connection != null) { + connection.close(); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java index 1e87e6e1121..e017bccd33d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java @@ -100,7 +100,7 @@ public class TestCloneSnapshotFromClient { // take an empty snapshot admin.snapshot(emptySnapshot, tableName); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); try { // enable table and insert data admin.enableTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index c6b58300bf3..5dd691a21f1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -182,8 +182,7 @@ public class TestFromClientSide { HTableDescriptor desc = new HTableDescriptor(TABLENAME); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); - Configuration c = TEST_UTIL.getConfiguration(); - Table h = new HTable(c, TABLENAME); + Table h = TEST_UTIL.getConnection().getTable(TABLENAME); long ts = System.currentTimeMillis(); Put p = new Put(T1, ts); @@ -415,7 +414,7 @@ public class TestFromClientSide { putRows(ht, 3, value2, keyPrefix1); putRows(ht, 3, value2, keyPrefix2); putRows(ht, 3, value2, keyPrefix3); - Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE); + Table table = TEST_UTIL.getConnection().getTable(TABLE); System.out.println("Checking values for key: " + keyPrefix1); assertEquals("Got back incorrect number of rows from scan", 3, getNumberOfRows(keyPrefix1, value2, table)); @@ -641,8 +640,8 @@ public class TestFromClientSide { private Map splitTable(final HTable t) throws IOException, InterruptedException { // Split this table in two. - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); - admin.split(t.getTableName()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); + admin.split(t.getName()); admin.close(); Map regions = waitOnSplit(t); assertTrue(regions.size() > 1); @@ -1748,7 +1747,7 @@ public class TestFromClientSide { @Test public void testDeleteFamilyVersion() throws Exception { - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion"); byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1); @@ -1793,7 +1792,7 @@ public class TestFromClientSide { byte [][] VALUES = makeN(VALUE, 5); long [] ts = {1000, 2000, 3000, 4000, 5000}; - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5); Put put = null; Result result = null; @@ -3633,7 +3632,7 @@ public class TestFromClientSide { TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction"); Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); // Write a column with values at timestamp 1, 2 and 3 byte[] row = Bytes.toBytes("row2"); @@ -3695,7 +3694,7 @@ public class TestFromClientSide { String tableName = "testMajorCompactionBetweenTwoUpdates"; byte [] TABLE = Bytes.toBytes(tableName); Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); // Write a column with values at timestamp 1, 2 and 3 byte[] row = Bytes.toBytes("row3"); @@ -4104,7 +4103,7 @@ public class TestFromClientSide { for (int i = 0; i < tables.length; i++) { TEST_UTIL.createTable(tables[i], FAMILY); } - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); HTableDescriptor[] ts = admin.listTables(); HashSet result = new HashSet(ts.length); Collections.addAll(result, ts); @@ -4196,7 +4195,7 @@ public class TestFromClientSide { a.put(put); // open a new connection to A and a connection to b - Table newA = new HTable(TEST_UTIL.getConfiguration(), tableAname); + Table newA = TEST_UTIL.getConnection().getTable(tableAname); // copy data from A to B Scan scan = new Scan(); @@ -4216,7 +4215,7 @@ public class TestFromClientSide { } // Opening a new connection to A will cause the tables to be reloaded - Table anotherA = new HTable(TEST_UTIL.getConfiguration(), tableAname); + Table anotherA = TEST_UTIL.getConnection().getTable(tableAname); Get get = new Get(ROW); get.addFamily(HConstants.CATALOG_FAMILY); anotherA.get(get); @@ -4226,7 +4225,7 @@ public class TestFromClientSide { // to be reloaded. // Test user metadata - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); // make a modifiable descriptor HTableDescriptor desc = new HTableDescriptor(a.getTableDescriptor()); // offline the table @@ -4984,12 +4983,9 @@ public class TestFromClientSide { public void testScanMetrics() throws Exception { TableName TABLENAME = TableName.valueOf("testScanMetrics"); - Configuration conf = TEST_UTIL.getConfiguration(); - TEST_UTIL.createTable(TABLENAME, FAMILY); - // Set up test table: // Create table: - HTable ht = new HTable(conf, TABLENAME); + HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY); // Create multiple regions for this table int numOfRegions = TEST_UTIL.createMultiRegions(ht, FAMILY); @@ -5203,7 +5199,7 @@ public class TestFromClientSide { byte [] family1 = Bytes.toBytes("f1"); byte [] family2 = Bytes.toBytes("f2"); try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10); - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration())) { + Admin admin = TEST_UTIL.getHBaseAdmin()) { Map regionsMap = table.getRegionLocations(); assertEquals(1, regionsMap.size()); HRegionInfo regionInfo = regionsMap.keySet().iterator().next(); 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 2729e970351..680dcfbb274 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 @@ -126,7 +126,7 @@ public class TestFromClientSide3 { int sfCount = sf.size(); // TODO: replace this api with a synchronous flush after HBASE-2949 - admin.flush(table.getTableName()); + admin.flush(table.getName()); // synchronously poll wait for a new storefile to appear (flush happened) while (ProtobufUtil.getStoreFiles( @@ -408,7 +408,7 @@ public class TestFromClientSide3 { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("test"))); desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); Put put = new Put(ROW_BYTES); put.add(FAMILY, COL_QUAL, VAL_BYTES); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java index 82a5c76ddbe..a86e4fa876e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java @@ -349,7 +349,8 @@ public class TestHCM { c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt); - final HTable table = new HTable(c2, tableName); + Connection connection = ConnectionFactory.createConnection(c2); + final HTable table = (HTable) connection.getTable(tableName); Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); @@ -409,6 +410,7 @@ public class TestHCM { }); table.close(); + connection.close(); Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null); TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true); } @@ -429,7 +431,8 @@ public class TestHCM { c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed c2.setInt(RpcClient.IDLE_TIME, idleTime); - final Table table = new HTable(c2, tableName); + Connection connection = ConnectionFactory.createConnection(c2); + final Table table = connection.getTable(tableName); Put put = new Put(ROW); put.add(FAM_NAM, ROW, ROW); @@ -465,15 +468,17 @@ public class TestHCM { LOG.info("we're done - time will change back"); table.close(); + + connection.close(); EnvironmentEdgeManager.reset(); TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true); } - /** - * Test that the connection to the dead server is cut immediately when we receive the - * notification. - * @throws Exception - */ + /** + * Test that the connection to the dead server is cut immediately when we receive the + * notification. + * @throws Exception + */ @Test public void testConnectionCut() throws Exception { if (!isJavaOk){ @@ -492,7 +497,8 @@ public class TestHCM { c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000); - HTable table = new HTable(c2, tableName); + final Connection connection = ConnectionFactory.createConnection(c2); + final HTable table = (HTable) connection.getTable(tableName); Put p = new Put(FAM_NAM); p.add(FAM_NAM, FAM_NAM, FAM_NAM); @@ -538,6 +544,7 @@ public class TestHCM { } table.close(); + connection.close(); } protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false); @@ -597,7 +604,8 @@ public class TestHCM { TEST_UTIL.createTable(TABLE_NAME, FAM_NAM).close(); Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - HTable table = new HTable(conf, TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + final HTable table = (HTable) connection.getTable(TABLE_NAME); TEST_UTIL.createMultiRegions(table, FAM_NAM); TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); @@ -773,6 +781,7 @@ public class TestHCM { TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER); table.close(); + connection.close(); } /** @@ -1240,7 +1249,7 @@ public class TestHCM { try { c1 = ConnectionManager.getConnectionInternal(config); LOG.info("HTable connection " + i + " " + c1); - Table table = new HTable(config, TABLE_NAME4, pool); + Table table = c1.getTable(TABLE_NAME4, pool); table.close(); LOG.info("HTable connection " + i + " closed " + c1); } catch (Exception e) { 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 47bb569dfe5..03d310f835d 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 @@ -170,7 +170,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithGet() throws Exception { LOG.info("test=testBatchWithGet"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); // load test data List puts = constructPutRequests(); @@ -209,7 +209,7 @@ public class TestMultiParallel { @Test public void testBadFam() throws Exception { LOG.info("test=testBadFam"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); List actions = new ArrayList(); Put p = new Put(Bytes.toBytes("row1")); @@ -262,7 +262,7 @@ public class TestMultiParallel { private void doTestFlushCommits(boolean doAbort) throws Exception { // Load the data LOG.info("get new table"); - HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); table.setAutoFlushTo(false); table.setWriteBufferSize(10 * 1024 * 1024); @@ -362,7 +362,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithDelete() throws Exception { LOG.info("test=testBatchWithDelete"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); // Load some data List puts = constructPutRequests(); @@ -391,7 +391,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testHTableDeleteWithList() throws Exception { LOG.info("test=testHTableDeleteWithList"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); // Load some data List puts = constructPutRequests(); @@ -420,7 +420,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithManyColsInOneRowGetAndPut() throws Exception { LOG.info("test=testBatchWithManyColsInOneRowGetAndPut"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); List puts = new ArrayList(); for (int i = 0; i < 100; i++) { @@ -461,7 +461,7 @@ public class TestMultiParallel { final byte[] QUAL2 = Bytes.toBytes("qual2"); final byte[] QUAL3 = Bytes.toBytes("qual3"); final byte[] QUAL4 = Bytes.toBytes("qual4"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); Delete d = new Delete(ONE_ROW); table.delete(d); Put put = new Put(ONE_ROW); @@ -583,7 +583,7 @@ public class TestMultiParallel { @Test(timeout=300000) public void testBatchWithMixedActions() throws Exception { LOG.info("test=testBatchWithMixedActions"); - Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); // Load some data to start Object[] results = table.batch(constructPutRequests()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index ca1254df4d3..6d98c52486b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -214,7 +214,7 @@ public class TestReplicaWithCluster { } HTU.getHBaseCluster().stopMaster(0); - Admin admin = new HBaseAdmin(HTU.getConfiguration()); + Admin admin = HTU.getHBaseAdmin(); nHdt =admin.getTableDescriptor(hdt.getTableName()); Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()), bHdt.getColumnFamilies().length + 1, nHdt.getColumnFamilies().length); @@ -255,7 +255,7 @@ public class TestReplicaWithCluster { Put p = new Put(row); p.add(row, row, row); - final Table table = new HTable(HTU.getConfiguration(), hdt.getTableName()); + final Table table = HTU.getConnection().getTable(hdt.getTableName()); table.put(p); HTU.getHBaseAdmin().flush(table.getName()); @@ -279,7 +279,7 @@ public class TestReplicaWithCluster { table.close(); LOG.info("stale get on the first cluster done. Now for the second."); - final Table table2 = new HTable(HTU.getConfiguration(), hdt.getTableName()); + final Table table2 = HTU.getConnection().getTable(hdt.getTableName()); Waiter.waitFor(HTU.getConfiguration(), 1000, new Waiter.Predicate() { @Override public boolean evaluate() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java index 5bbd8bec57a..a5dce028bca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java @@ -111,7 +111,7 @@ public class TestRestoreSnapshotFromClient { // take an empty snapshot admin.snapshot(emptySnapshot, tableName); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); // enable table and insert data admin.enableTable(tableName); SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY); @@ -175,7 +175,7 @@ public class TestRestoreSnapshotFromClient { public void testRestoreSchemaChange() throws Exception { byte[] TEST_FAMILY2 = Bytes.toBytes("cf2"); - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); // Add one column family and put some data in it admin.disableTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java index 1740cc88770..018bdc4044d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcControllerFactory.java @@ -131,7 +131,8 @@ public class TestRpcControllerFactory { // change one of the connection properties so we get a new HConnection with our configuration conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT + 1); - Table table = new HTable(conf, name); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(name); table.setAutoFlushTo(false); byte[] row = Bytes.toBytes("row"); Put p = new Put(row); @@ -187,6 +188,7 @@ public class TestRpcControllerFactory { counter = doScan(table, scanInfo, counter); table.close(); + connection.close(); } int doScan(Table table, Scan scan, int expectedCount) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java index b46312f91cf..f9c80308448 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannerTimeout.java @@ -101,8 +101,7 @@ public class TestScannerTimeout { LOG.info("START ************ test2481"); Scan scan = new Scan(); scan.setCaching(1); - Table table = - new HTable(new Configuration(TEST_UTIL.getConfiguration()), TABLE_NAME); + Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME); ResultScanner r = table.getScanner(scan); int count = 0; try { @@ -141,7 +140,9 @@ public class TestScannerTimeout { // this new table Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); - Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME); + + Connection connection = ConnectionFactory.createConnection(conf); + Table higherScanTimeoutTable = connection.getTable(TABLE_NAME); ResultScanner r = higherScanTimeoutTable.getScanner(scan); // This takes way less than SCANNER_TIMEOUT*100 rs.abort("die!"); @@ -149,6 +150,7 @@ public class TestScannerTimeout { assertEquals(NB_ROWS, results.length); r.close(); higherScanTimeoutTable.close(); + connection.close(); LOG.info("END ************ test2772"); } @@ -175,7 +177,8 @@ public class TestScannerTimeout { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt( HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT*100); - Table table = new HTable(conf, TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(TABLE_NAME); LOG.info("START ************ TEST3686A---22"); ResultScanner r = table.getScanner(scan); @@ -193,6 +196,7 @@ public class TestScannerTimeout { assertEquals(NB_ROWS, count); r.close(); table.close(); + connection.close(); LOG.info("************ END TEST3686A"); } @@ -214,7 +218,8 @@ public class TestScannerTimeout { // this new table Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); conf.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, SCANNER_TIMEOUT * 100); - Table higherScanTimeoutTable = new HTable(conf, TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table higherScanTimeoutTable = connection.getTable(TABLE_NAME); ResultScanner r = higherScanTimeoutTable.getScanner(scan); int count = 1; r.next(); @@ -226,6 +231,7 @@ public class TestScannerTimeout { assertEquals(NB_ROWS, count); r.close(); higherScanTimeoutTable.close(); + connection.close(); LOG.info("END ************ END test3686b"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java index b4c6b3bb11a..004ebee8330 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java @@ -18,19 +18,17 @@ package org.apache.hadoop.hbase.client; -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; -import org.apache.hadoop.hbase.TableName; 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.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; @@ -45,6 +43,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.List; + /** * Test to verify that the cloned table is independent of the table from which it was cloned */ @@ -211,7 +211,7 @@ public class TestSnapshotCloneIndependence { TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName); admin.cloneSnapshot(snapshotName, cloneTableName); - try (Table clonedTable = new HTable(UTIL.getConfiguration(), cloneTableName)){ + try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) { final int clonedTableRowCount = UTIL.countRows(clonedTable); Assert.assertEquals( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index c17da6d4b55..079a5888746 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -149,7 +149,7 @@ public class TestSnapshotFromClient { SnapshotTestingUtils.assertNoSnapshots(admin); // put some stuff in the table - HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME); + Table table = UTIL.getConnection().getTable(TABLE_NAME); UTIL.loadTable(table, TEST_FAM); table.close(); @@ -185,7 +185,7 @@ public class TestSnapshotFromClient { SnapshotTestingUtils.assertNoSnapshots(admin); // put some stuff in the table - HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME); + Table table = UTIL.getConnection().getTable(TABLE_NAME); UTIL.loadTable(table, TEST_FAM, false); LOG.debug("FS state before disable:"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java index 6f39d3b904e..e3c7ef94991 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java @@ -171,7 +171,7 @@ public class TestSnapshotMetadata { assertTrue(htd.getConfiguration().size() > 0); admin.createTable(htd); - Table original = new HTable(UTIL.getConfiguration(), originalTableName); + Table original = UTIL.getConnection().getTable(originalTableName); originalTableName = TableName.valueOf(sourceTableNameAsString); originalTableDescriptor = admin.getTableDescriptor(originalTableName); originalTableDescription = originalTableDescriptor.toStringCustomizedValues(); @@ -201,7 +201,7 @@ public class TestSnapshotMetadata { familiesList, snapshotNameAsString, rootDir, fs, /* onlineSnapshot= */ false); admin.cloneSnapshot(snapshotName, clonedTableName); - Table clonedTable = new HTable(UTIL.getConfiguration(), clonedTableName); + Table clonedTable = UTIL.getConnection().getTable(clonedTableName); HTableDescriptor cloneHtd = admin.getTableDescriptor(clonedTableName); assertEquals( originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString), @@ -265,7 +265,7 @@ public class TestSnapshotMetadata { List familiesWithDataList = new ArrayList(); List emptyFamiliesList = new ArrayList(); if (addData) { - HTable original = new HTable(UTIL.getConfiguration(), originalTableName); + Table original = UTIL.getConnection().getTable(originalTableName); UTIL.loadTable(original, familyForUpdate); // family arbitrarily chosen original.close(); @@ -310,7 +310,7 @@ public class TestSnapshotMetadata { admin.enableTable(originalTableName); // verify that the descrption is reverted - Table original = new HTable(UTIL.getConfiguration(), originalTableName); + Table original = UTIL.getConnection().getTable(originalTableName); try { assertTrue(originalTableDescriptor.equals(admin.getTableDescriptor(originalTableName))); assertTrue(originalTableDescriptor.equals(original.getTableDescriptor())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java index 1e3d1cfda42..0f0baffc0eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableSnapshotScanner.java @@ -92,7 +92,7 @@ public class TestTableSnapshotScanner { Admin admin = util.getHBaseAdmin(); // put some stuff in the table - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); util.loadTable(table, FAMILIES); Path rootDir = FSUtils.getRootDir(util.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java index f4ad44c374f..96da03a8a2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java @@ -82,7 +82,7 @@ public class TestConstraint { Constraints.add(desc, CheckWasRunConstraint.class); util.getHBaseAdmin().createTable(desc); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); try { // test that we don't fail on a valid put Put put = new Put(row1); @@ -114,7 +114,7 @@ public class TestConstraint { Constraints.add(desc, AllFailConstraint.class); util.getHBaseAdmin().createTable(desc); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); // test that we do fail on violation Put put = new Put(row1); @@ -157,7 +157,7 @@ public class TestConstraint { Constraints.disableConstraint(desc, AllFailConstraint.class); util.getHBaseAdmin().createTable(desc); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); try { // test that we don't fail because its disabled Put put = new Put(row1); @@ -189,7 +189,7 @@ public class TestConstraint { Constraints.disable(desc); util.getHBaseAdmin().createTable(desc); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); try { // test that we do fail on violation Put put = new Put(row1); @@ -221,7 +221,7 @@ public class TestConstraint { CheckWasRunConstraint.wasRun = false; util.getHBaseAdmin().createTable(desc); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); // test that we do fail on violation Put put = new Put(row1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java index c99a1f6d65a..a5ee4c195c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.TreeMap; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -86,14 +87,14 @@ public class TestBatchCoprocessorEndpoint { conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); - Admin admin = new HBaseAdmin(conf); + Admin admin = util.getHBaseAdmin(); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); admin.close(); - Table table = new HTable(conf, TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); @@ -109,7 +110,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationNullResponse() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); ColumnAggregationWithNullResponseProtos.SumRequest.Builder builder = ColumnAggregationWithNullResponseProtos.SumRequest .newBuilder(); @@ -162,7 +163,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationWithReturnValue() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length - 1]); int sumResult = 0; @@ -198,7 +199,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregation() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length - 1]); int sumResult = 0; @@ -231,7 +232,7 @@ public class TestBatchCoprocessorEndpoint { @Test public void testAggregationWithErrors() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); final Map results = Collections.synchronizedMap( new TreeMap( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index 64732b0d95f..c04499d3322 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -94,14 +94,14 @@ public class TestCoprocessorEndpoint { conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); - Admin admin = new HBaseAdmin(conf); + Admin admin = util.getHBaseAdmin(); HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); util.waitUntilAllRegionsAssigned(TEST_TABLE); admin.close(); - Table table = new HTable(conf, TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); @@ -140,7 +140,7 @@ public class TestCoprocessorEndpoint { @Test public void testAggregation() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], ROWS[ROWS.length-1]); int sumResult = 0; @@ -174,7 +174,7 @@ public class TestCoprocessorEndpoint { @Test public void testCoprocessorService() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + HTable table = (HTable) util.getConnection().getTable(TEST_TABLE); NavigableMap regions = table.getRegionLocations(); final TestProtos.EchoRequestProto request = @@ -248,7 +248,7 @@ public class TestCoprocessorEndpoint { @Test public void testCoprocessorServiceNullResponse() throws Throwable { - HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + HTable table = (HTable) util.getConnection().getTable(TEST_TABLE); NavigableMap regions = table.getRegionLocations(); final TestProtos.EchoRequestProto request = @@ -299,7 +299,7 @@ public class TestCoprocessorEndpoint { Configuration configuration = new Configuration(util.getConfiguration()); // Make it not retry forever configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - Table table = new HTable(configuration, TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); try { CoprocessorRpcChannel protocol = table.coprocessorService(ROWS[0]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java index c5af6d570f7..4c3594b9467 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java @@ -18,23 +18,12 @@ */ package org.apache.hadoop.hbase.coprocessor; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.Collections; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -50,6 +39,16 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + /** * Test that a coprocessor can open a connection and write to another table, inside a hook. */ @@ -161,13 +160,13 @@ public class TestOpenTableInCoprocessor { admin.createTable(primary); admin.createTable(other); - Table table = new HTable(UTIL.getConfiguration(), TableName.valueOf("primary")); + Table table = UTIL.getConnection().getTable(TableName.valueOf("primary")); Put p = new Put(new byte[] { 'a' }); p.add(family, null, new byte[] { 'a' }); table.put(p); table.close(); - Table target = new HTable(UTIL.getConfiguration(), otherTable); + Table target = UTIL.getConnection().getTable(otherTable); assertTrue("Didn't complete update to target table!", completeCheck[0]); assertEquals("Didn't find inserted row", 1, getKeyValueCount(target)); target.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java index 3e41859898a..a02758dacff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java @@ -93,7 +93,7 @@ public class TestRegionObserverBypass { */ @Test public void testSimple() throws Exception { - Table t = new HTable(util.getConfiguration(), tableName); + Table t = util.getConnection().getTable(tableName); Put p = new Put(row1); p.add(test,dummy,dummy); // before HBASE-4331, this would throw an exception @@ -112,7 +112,7 @@ public class TestRegionObserverBypass { //previous deletes will eclipse successive puts having the same timestamp EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge()); - Table t = new HTable(util.getConfiguration(), tableName); + Table t = util.getConnection().getTable(tableName); List puts = new ArrayList(); Put p = new Put(row1); p.add(dummy,dummy,dummy); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index 0f161d9ed13..6895fbea084 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -340,7 +340,7 @@ public class TestRegionObserverInterface { new Boolean[] {false, false, false, false} ); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); Put put = new Put(ROW); put.add(A, A, A); table.put(put); @@ -390,7 +390,7 @@ public class TestRegionObserverInterface { new Boolean[] {false, false} ); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); Put put = new Put(ROW); put.add(A, A, A); table.put(put); @@ -497,7 +497,7 @@ public class TestRegionObserverInterface { htd.addCoprocessor(EvenOnlyCompactor.class.getName()); admin.createTable(htd); - Table table = new HTable(util.getConfiguration(), compactTable); + Table table = util.getConnection().getTable(compactTable); for (long i=1; i<=10; i++) { byte[] iBytes = Bytes.toBytes(i); Put put = new Put(iBytes); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 87420665724..88b001d5daf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -259,7 +259,7 @@ public class TestRegionObserverScannerOpenHook { Admin admin = UTIL.getHBaseAdmin(); admin.createTable(desc); - Table table = new HTable(conf, desc.getTableName()); + Table table = UTIL.getConnection().getTable(desc.getTableName()); // put a row and flush it to disk Put put = new Put(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java index df85004cd45..7ae6787b27e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorEndpoint.java @@ -73,7 +73,7 @@ public class TestRegionServerCoprocessorEndpoint { new BlockingRpcCallback(); DummyRegionServerEndpointProtos.DummyService service = ProtobufUtil.newServiceStub(DummyRegionServerEndpointProtos.DummyService.class, - new HBaseAdmin(CONF).coprocessorService(serverName)); + TEST_UTIL.getHBaseAdmin().coprocessorService(serverName)); service.dummyCall(controller, DummyRegionServerEndpointProtos.DummyRequest.getDefaultInstance(), rpcCallback); assertEquals(DUMMY_VALUE, rpcCallback.get().getValue()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java index 0c30bb2695e..2e6eabc2648 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java @@ -76,7 +76,7 @@ public class TestRegionServerObserver { // Start the cluster HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf); TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS); - Admin admin = new HBaseAdmin(conf); + Admin admin = TEST_UTIL.getHBaseAdmin(); try { MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); HRegionServer regionServer = cluster.getRegionServer(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java index 76290fbbf4a..982b9776ef2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java @@ -37,6 +37,9 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; +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.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.ScannerCallable; @@ -58,13 +61,13 @@ import org.junit.experimental.categories.Category; @Category({FilterTests.class, MediumTests.class}) public class FilterTestingCluster { private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Configuration conf = null; - private static HBaseAdmin admin = null; - private static List createdTables = new ArrayList<>(); + private static Connection connection; + private static Admin admin = null; + private static List createdTables = new ArrayList<>(); - protected static void createTable(String tableName, String columnFamilyName) { + protected static void createTable(TableName tableName, String columnFamilyName) { assertNotNull("HBaseAdmin is not initialized successfully.", admin); - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); + HTableDescriptor desc = new HTableDescriptor(tableName); HColumnDescriptor colDef = new HColumnDescriptor(Bytes.toBytes(columnFamilyName)); desc.addFamily(colDef); @@ -77,15 +80,15 @@ public class FilterTestingCluster { } } - protected static Table openTable(String tableName) throws IOException { - Table table = new HTable(conf, tableName); + protected static Table openTable(TableName tableName) throws IOException { + Table table = connection.getTable(tableName); assertTrue("Fail to create the table", admin.tableExists(tableName)); return table; } private static void deleteTables() { if (admin != null) { - for (String tableName: createdTables){ + for (TableName tableName: createdTables){ try { if (admin.tableExists(tableName)) { admin.disableTable(tableName); @@ -99,10 +102,11 @@ public class FilterTestingCluster { } private static void initialize(Configuration conf) { - FilterTestingCluster.conf = HBaseConfiguration.create(conf); - FilterTestingCluster.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + conf = HBaseConfiguration.create(conf); + conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); try { - admin = new HBaseAdmin(conf); + connection = ConnectionFactory.createConnection(conf); + admin = connection.getAdmin(); } catch (MasterNotRunningException e) { assertNull("Master is not running", e); } catch (ZooKeeperConnectionException e) { @@ -124,7 +128,7 @@ public class FilterTestingCluster { @AfterClass public static void tearDown() throws Exception { deleteTables(); + connection.close(); TEST_UTIL.shutdownMiniCluster(); } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java index 142b15a9cde..78a4d1fe5cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java @@ -30,6 +30,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -50,7 +51,7 @@ public class TestFilterWithScanLimits extends FilterTestingCluster { private static final Log LOG = LogFactory .getLog(TestFilterWithScanLimits.class); - private static final String tableName = "scanWithLimit"; + private static final TableName tableName = TableName.valueOf("scanWithLimit"); private static final String columnFamily = "f1"; @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java index 1cffe1db8aa..8ce0b760a81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java @@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -66,6 +68,7 @@ public class TestFilterWrapper { private static Configuration conf = null; private static HBaseAdmin admin = null; private static TableName name = TableName.valueOf("test"); + private static Connection connection; @Test public void testFilterWrapper() { @@ -84,7 +87,7 @@ public class TestFilterWrapper { FilterList filter = new FilterList(fs); scan.setFilter(filter); - Table table = new HTable(conf, name); + Table table = connection.getTable(name); ResultScanner scanner = table.getScanner(scan); // row2 (c1-c4) and row3(c1-c4) are returned @@ -111,7 +114,7 @@ public class TestFilterWrapper { private static void prepareData() { try { - Table table = new HTable(TestFilterWrapper.conf, name); + Table table = connection.getTable(name); assertTrue("Fail to create the table", admin.tableExists(name)); List puts = new ArrayList(); @@ -173,7 +176,8 @@ public class TestFilterWrapper { TestFilterWrapper.conf = HBaseConfiguration.create(conf); TestFilterWrapper.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); try { - admin = new HBaseAdmin(conf); + connection = ConnectionFactory.createConnection(TestFilterWrapper.conf); + admin = TEST_UTIL.getHBaseAdmin(); } catch (MasterNotRunningException e) { assertNull("Master is not running", e); } catch (ZooKeeperConnectionException e) { @@ -187,7 +191,6 @@ public class TestFilterWrapper { @BeforeClass public static void setUp() throws Exception { - Configuration config = TEST_UTIL.getConfiguration(); TEST_UTIL.startMiniCluster(1); initialize(TEST_UTIL.getConfiguration()); } @@ -195,6 +198,7 @@ public class TestFilterWrapper { @AfterClass public static void tearDown() throws Exception { deleteTable(); + connection.close(); TEST_UTIL.shutdownMiniCluster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java index 100f26d4cfd..3be10ec3333 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestScanRowPrefix.java @@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.codec.binary.Hex; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -48,7 +49,7 @@ public class TestScanRowPrefix extends FilterTestingCluster { @Test public void testPrefixScanning() throws IOException { - String tableName = "prefixScanning"; + TableName tableName = TableName.valueOf("prefixScanning"); createTable(tableName,"F"); Table table = openTable(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java index fdbef44c950..918e0f18d81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java @@ -16,15 +16,6 @@ */ package org.apache.hadoop.hbase.io.encoding; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -41,7 +32,6 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; @@ -55,6 +45,15 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertTrue; + /** * Tests changing data block encoding settings of a column family. */ @@ -127,7 +126,7 @@ public class TestChangingEncoding { + "_col" + j); } - static void writeTestDataBatch(Configuration conf, TableName tableName, + static void writeTestDataBatch(TableName tableName, int batchId) throws Exception { LOG.debug("Writing test data batch " + batchId); List puts = new ArrayList<>(); @@ -146,10 +145,10 @@ public class TestChangingEncoding { } } - static void verifyTestDataBatch(Configuration conf, TableName tableName, + static void verifyTestDataBatch(TableName tableName, int batchId) throws Exception { LOG.debug("Verifying test data batch " + batchId); - Table table = new HTable(conf, tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) { Get get = new Get(getRowKey(batchId, i)); Result result = table.get(get); @@ -162,13 +161,13 @@ public class TestChangingEncoding { } private void writeSomeNewData() throws Exception { - writeTestDataBatch(conf, tableName, numBatchesWritten); + writeTestDataBatch(tableName, numBatchesWritten); ++numBatchesWritten; } private void verifyAllData() throws Exception { for (int i = 0; i < numBatchesWritten; ++i) { - verifyTestDataBatch(conf, tableName, i); + verifyTestDataBatch(tableName, i); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java index 26183ac7fd0..d6088ecb216 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java @@ -68,14 +68,14 @@ public class TestLoadAndSwitchEncodeOnDisk extends @Test(timeout=TIMEOUT_MS) public void loadTest() throws Exception { - HBaseAdmin admin = new HBaseAdmin(conf); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); compression = Compression.Algorithm.GZ; // used for table setup super.loadTest(); HColumnDescriptor hcd = getColumnDesc(admin); System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n"); - HTable t = new HTable(this.conf, TABLE); + HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLE); assertAllOnLine(t); admin.disableTable(TABLE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java index 107837e6e21..35e6ae5aeeb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java @@ -26,6 +26,7 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.HTable; @@ -73,7 +74,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase { } @Override - protected void runTestOnTable(HTable table) throws IOException { + protected void runTestOnTable(Table table) throws IOException { JobConf jobConf = null; try { LOG.info("Before map/reduce startup"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java index 1f8ccae75ae..43e29adb666 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatTestBase.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; @@ -152,7 +153,7 @@ public abstract class TableSnapshotInputFormatTestBase { Admin admin = util.getHBaseAdmin(); // put some stuff in the table - HTable table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); util.loadTable(table, FAMILIES); Path rootDir = FSUtils.getRootDir(util.getConfiguration()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index c4478951147..92a7b42faa1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.Arrays; @@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PerformanceEvaluation; import org.apache.hadoop.hbase.TableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HRegionLocator; import org.apache.hadoop.hbase.client.HTable; @@ -378,7 +380,7 @@ public class TestHFileOutputFormat { try { util.startMiniCluster(); Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad"); - admin = new HBaseAdmin(conf); + admin = util.getHBaseAdmin(); HTable table = util.createTable(TABLE_NAME, FAMILIES); assertEquals("Should start with empty table", 0, util.countRows(table)); @@ -408,7 +410,7 @@ public class TestHFileOutputFormat { // handle the split case if (shouldChangeRegions) { LOG.info("Changing regions in table"); - admin.disableTable(table.getTableName()); + admin.disableTable(table.getName()); while(util.getMiniHBaseCluster().getMaster().getAssignmentManager(). getRegionStates().isRegionsInTransition()) { Threads.sleep(200); @@ -417,9 +419,9 @@ public class TestHFileOutputFormat { byte[][] newStartKeys = generateRandomStartKeys(15); util.createMultiRegions( util.getConfiguration(), table, FAMILIES[0], newStartKeys); - admin.enableTable(table.getTableName()); + admin.enableTable(table.getName()); while (table.getRegionLocations().size() != 15 || - !admin.isTableAvailable(table.getTableName())) { + !admin.isTableAvailable(table.getName())) { Thread.sleep(200); LOG.info("Waiting for new region assignment to happen"); } @@ -967,7 +969,7 @@ public class TestHFileOutputFormat { util.startMiniCluster(); Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction"); final FileSystem fs = util.getDFSCluster().getFileSystem(); - HBaseAdmin admin = new HBaseAdmin(conf); + Admin admin = util.getHBaseAdmin(); HTable table = util.createTable(TABLE_NAME, FAMILIES); assertEquals("Should start with empty table", 0, util.countRows(table)); @@ -982,7 +984,7 @@ public class TestHFileOutputFormat { Put p = new Put(Bytes.toBytes("test")); p.add(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1")); table.put(p); - admin.flush(TABLE_NAME.getName()); + admin.flush(TABLE_NAME); assertEquals(1, util.countRows(table)); quickPoll(new Callable() { public Boolean call() throws Exception { @@ -1008,7 +1010,7 @@ public class TestHFileOutputFormat { assertEquals(2, fs.listStatus(storePath).length); // minor compactions shouldn't get rid of the file - admin.compact(TABLE_NAME.getName()); + admin.compact(TABLE_NAME); try { quickPoll(new Callable() { public Boolean call() throws Exception { @@ -1021,7 +1023,7 @@ public class TestHFileOutputFormat { } // a major compaction should work though - admin.majorCompact(TABLE_NAME.getName()); + admin.majorCompact(TABLE_NAME); quickPoll(new Callable() { public Boolean call() throws Exception { return fs.listStatus(storePath).length == 1; @@ -1063,7 +1065,7 @@ public class TestHFileOutputFormat { admin.enableTable(tname); } else if ("incremental".equals(args[0])) { TableName tname = TableName.valueOf(args[1]); - HTable table = new HTable(conf, tname); + HTable table = (HTable) util.getConnection().getTable(tname); Path outDir = new Path("incremental-out"); runIncrementalPELoad(conf, table, outDir); } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index e828fa65394..935d462d05d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -255,7 +255,7 @@ public class TestImportExport { .setMaxVersions(1) ); UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table t = UTIL.getConnection().getTable(desc.getTableName()); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -286,7 +286,7 @@ public class TestImportExport { .setKeepDeletedCells(true) ); UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table t = UTIL.getConnection().getTable(desc.getTableName()); Put p = new Put(ROW1); p.add(FAMILYA, QUAL, now, QUAL); @@ -318,7 +318,7 @@ public class TestImportExport { ); UTIL.getHBaseAdmin().createTable(desc); t.close(); - t = new HTable(UTIL.getConfiguration(), desc.getTableName()); + t = UTIL.getConnection().getTable(desc.getTableName()); args = new String[] { IMPORT_TABLE, FQ_OUTPUT_DIR @@ -344,14 +344,16 @@ public class TestImportExport { @Test public void testWithMultipleDeleteFamilyMarkersOfSameRowSameFamily() throws Exception { - String EXPORT_TABLE = "exportWithMultipleDeleteFamilyMarkersOfSameRowSameFamily"; - HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); + TableName EXPORT_TABLE = + TableName.valueOf("exportWithMultipleDeleteFamilyMarkersOfSameRowSameFamily"); + HTableDescriptor desc = new HTableDescriptor(EXPORT_TABLE); desc.addFamily(new HColumnDescriptor(FAMILYA) .setMaxVersions(5) .setKeepDeletedCells(true) ); UTIL.getHBaseAdmin().createTable(desc); - HTable exportT = new HTable(UTIL.getConfiguration(), EXPORT_TABLE); + + Table exportT = UTIL.getConnection().getTable(EXPORT_TABLE); //Add first version of QUAL Put p = new Put(ROW1); @@ -373,8 +375,7 @@ public class TestImportExport { String[] args = new String[] { - "-D" + Export.RAW_SCAN + "=true", - EXPORT_TABLE, + "-D" + Export.RAW_SCAN + "=true", EXPORT_TABLE.getNameAsString(), FQ_OUTPUT_DIR, "1000", // max number of key versions per key to export }; @@ -387,8 +388,8 @@ public class TestImportExport { .setKeepDeletedCells(true) ); UTIL.getHBaseAdmin().createTable(desc); - - HTable importT = new HTable(UTIL.getConfiguration(), IMPORT_TABLE); + + Table importT = UTIL.getConnection().getTable(TableName.valueOf(IMPORT_TABLE)); args = new String[] { IMPORT_TABLE, FQ_OUTPUT_DIR @@ -429,7 +430,7 @@ public class TestImportExport { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE)); desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - Table exportTable = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table exportTable = UTIL.getConnection().getTable(desc.getTableName()); Put p1 = new Put(ROW1); p1.add(FAMILYA, QUAL, now, QUAL); @@ -454,7 +455,7 @@ public class TestImportExport { desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5)); UTIL.getHBaseAdmin().createTable(desc); - Table importTable = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table importTable = UTIL.getConnection().getTable(desc.getTableName()); args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + PrefixFilter.class.getName(), "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR, "1000" }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java index eddee5a12ad..8bd67718ade 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java @@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; 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.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.client.Durability; @@ -100,7 +102,6 @@ public class TestImportTSVWithOperationAttributes implements Configurable { conf.set("hbase.coprocessor.master.classes", OperationAttributesTestController.class.getName()); conf.set("hbase.coprocessor.region.classes", OperationAttributesTestController.class.getName()); util.startMiniCluster(); - Admin admin = new HBaseAdmin(util.getConfiguration()); util.startMiniMapReduceCluster(); } @@ -196,7 +197,8 @@ public class TestImportTSVWithOperationAttributes implements Configurable { int valueMultiplier, boolean dataAvailable) throws IOException { LOG.debug("Validating table."); - Table table = new HTable(conf, tableName); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -238,6 +240,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable { } } table.close(); + connection.close(); assertTrue(verified); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java index 0ca0f8ff449..6754ce981f0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java @@ -122,7 +122,6 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { // Wait for the labels table to become available util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); - Admin admin = new HBaseAdmin(util.getConfiguration()); util.startMiniMapReduceCluster(); } @@ -185,7 +184,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { private void issueDeleteAndVerifyData(TableName tableName) throws IOException { LOG.debug("Validating table after delete."); - Table table = new HTable(conf, tableName); + Table table = util.getConnection().getTable(tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -370,7 +369,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable { int valueMultiplier) throws IOException { LOG.debug("Validating table."); - Table table = new HTable(conf, tableName); + Table table = util.getConnection().getTable(tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java index 3844a64d5c4..9e2e4be5bf0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java @@ -43,6 +43,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -346,7 +348,8 @@ public class TestImportTsv implements Configurable { String family, int valueMultiplier) throws IOException { LOG.debug("Validating table."); - Table table = new HTable(conf, tableName); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName); boolean verified = false; long pause = conf.getLong("hbase.client.pause", 5 * 1000); int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); @@ -378,6 +381,7 @@ public class TestImportTsv implements Configurable { } } table.close(); + connection.close(); assertTrue(verified); } 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 fff02009257..d68d55c9a0d 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 @@ -255,7 +255,7 @@ public class TestLoadIncrementalHFiles { String [] args= {dir.toString(), tableName.toString()}; loader.run(args); - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); try { assertEquals(expectedRows, util.countRows(table)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java index 4d4043b5fed..e07353f80cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java @@ -242,7 +242,7 @@ public class TestLoadIncrementalHFilesSplitRecovery { assertEquals(htds.length, 1); Table t = null; try { - t = new HTable(util.getConfiguration(), table); + t = util.getConnection().getTable(table); Scan s = new Scan(); ResultScanner sr = t.getScanner(s); int i = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java index 20d577d680a..99684e9da20 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java @@ -126,11 +126,10 @@ public class TestMultithreadedTableMapper { @Test public void testMultithreadedTableMapper() throws IOException, InterruptedException, ClassNotFoundException { - runTestOnTable(new HTable(new Configuration(UTIL.getConfiguration()), - MULTI_REGION_TABLE_NAME)); + runTestOnTable(UTIL.getConnection().getTable(MULTI_REGION_TABLE_NAME)); } - private void runTestOnTable(HTable table) + private void runTestOnTable(Table table) throws IOException, InterruptedException, ClassNotFoundException { Job job = null; try { @@ -140,16 +139,16 @@ public class TestMultithreadedTableMapper { Scan scan = new Scan(); scan.addFamily(INPUT_FAMILY); TableMapReduceUtil.initTableMapperJob( - table.getTableName(), scan, + table.getName(), scan, MultithreadedTableMapper.class, ImmutableBytesWritable.class, Put.class, job); MultithreadedTableMapper.setMapperClass(job, ProcessContentsMapper.class); MultithreadedTableMapper.setNumberOfThreads(job, NUMBER_OF_THREADS); TableMapReduceUtil.initTableReducerJob( - Bytes.toString(table.getTableName()), + table.getName().getNameAsString(), IdentityTableReducer.class, job); FileOutputFormat.setOutputPath(job, new Path("test")); - LOG.info("Started " + table.getTableName()); + LOG.info("Started " + table.getName()); assertTrue(job.waitForCompletion(true)); LOG.info("After map/reduce completion"); // verify map-reduce results @@ -164,7 +163,7 @@ public class TestMultithreadedTableMapper { } private void verify(TableName tableName) throws IOException { - Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName); + Table table = UTIL.getConnection().getTable(tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java index 11a35f0174c..1dcbd2ab07d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduce.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; @@ -90,7 +91,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase { } } - protected void runTestOnTable(HTable table) throws IOException { + protected void runTestOnTable(Table table) throws IOException { Job job = null; try { LOG.info("Before map/reduce startup"); @@ -99,14 +100,14 @@ public class TestTableMapReduce extends TestTableMapReduceBase { Scan scan = new Scan(); scan.addFamily(INPUT_FAMILY); TableMapReduceUtil.initTableMapperJob( - Bytes.toString(table.getTableName()), scan, + table.getName().getNameAsString(), scan, ProcessContentsMapper.class, ImmutableBytesWritable.class, Put.class, job); TableMapReduceUtil.initTableReducerJob( - Bytes.toString(table.getTableName()), + table.getName().getNameAsString(), IdentityTableReducer.class, job); FileOutputFormat.setOutputPath(job, new Path("test")); - LOG.info("Started " + Bytes.toString(table.getTableName())); + LOG.info("Started " + table.getName().getNameAsString()); assertTrue(job.waitForCompletion(true)); LOG.info("After map/reduce completion"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java index c82d82ffb2b..b42966ceee8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java @@ -71,7 +71,7 @@ public abstract class TestTableMapReduceBase { /** * Handles API-specifics for setting up and executing the job. */ - protected abstract void runTestOnTable(HTable table) throws IOException; + protected abstract void runTestOnTable(Table table) throws IOException; @BeforeClass public static void beforeClass() throws Exception { @@ -95,7 +95,7 @@ public abstract class TestTableMapReduceBase { */ @Test public void testMultiRegionTable() throws IOException { - runTestOnTable(new HTable(UTIL.getConfiguration(), MULTI_REGION_TABLE_NAME)); + runTestOnTable(UTIL.getConnection().getTable(MULTI_REGION_TABLE_NAME)); } @Test @@ -103,7 +103,7 @@ public abstract class TestTableMapReduceBase { Configuration conf = new Configuration(UTIL.getConfiguration()); // force use of combiner for testing purposes conf.setInt("mapreduce.map.combine.minspills", 1); - runTestOnTable(new HTable(conf, MULTI_REGION_TABLE_NAME)); + runTestOnTable(UTIL.getConnection().getTable(MULTI_REGION_TABLE_NAME)); } /** @@ -134,7 +134,7 @@ public abstract class TestTableMapReduceBase { } protected void verify(TableName tableName) throws IOException { - Table table = new HTable(UTIL.getConfiguration(), tableName); + Table table = UTIL.getConnection().getTable(tableName); boolean verified = false; long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000); int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java index f13834855bb..03da1ed5c9c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java @@ -18,28 +18,25 @@ */ package org.apache.hadoop.hbase.mapreduce; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -49,13 +46,20 @@ import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -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; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + @Category({MapReduceTests.class, LargeTests.class}) public class TestTimeRangeMapRed { private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class); @@ -94,12 +98,7 @@ public class TestTimeRangeMapRed { @Before public void before() throws Exception { - this.admin = new HBaseAdmin(UTIL.getConfiguration()); - } - - @After - public void after() throws IOException { - this.admin.close(); + this.admin = UTIL.getHBaseAdmin(); } private static class ProcessTimeRangeMapper @@ -137,7 +136,8 @@ public class TestTimeRangeMapRed { public void setConf(Configuration configuration) { this.conf = configuration; try { - table = new HTable(HBaseConfiguration.create(conf), TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + table = connection.getTable(TABLE_NAME); } catch (IOException e) { e.printStackTrace(); } @@ -159,7 +159,7 @@ public class TestTimeRangeMapRed { put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false)); puts.add(put); } - Table table = new HTable(UTIL.getConfiguration(), desc.getTableName()); + Table table = UTIL.getConnection().getTable(desc.getTableName()); table.put(puts); runTestOnTable(); verify(table); @@ -204,7 +204,7 @@ public class TestTimeRangeMapRed { + "\t" + Bytes.toString(CellUtil.cloneQualifier(kv)) + "\t" + kv.getTimestamp() + "\t" + Bytes.toBoolean(CellUtil.cloneValue(kv))); org.junit.Assert.assertEquals(TIMESTAMP.get(kv.getTimestamp()), - (Boolean)Bytes.toBoolean(CellUtil.cloneValue(kv))); + Bytes.toBoolean(CellUtil.cloneValue(kv))); } } scanner.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java index fccff593e72..f482dfa837a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java @@ -205,7 +205,7 @@ public class TestAssignmentListener { assertEquals(0, listener.getCloseCount()); // Add some data - Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME); + Table table = TEST_UTIL.getConnection().getTable(TABLE_NAME); try { for (int i = 0; i < 10; ++i) { byte[] key = Bytes.toBytes("row-" + i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java index e6d08b976c3..d1769e8c240 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java @@ -181,7 +181,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -224,7 +224,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); final HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -436,7 +436,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -485,7 +485,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -531,7 +531,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -573,7 +573,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -658,7 +658,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -703,7 +703,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -776,7 +776,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -904,7 +904,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -945,7 +945,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -1024,7 +1024,7 @@ public class TestAssignmentManagerOnCluster { desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo( desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); @@ -1099,7 +1099,7 @@ public class TestAssignmentManagerOnCluster { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table)); desc.addFamily(new HColumnDescriptor(FAMILY)); admin.createTable(desc); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HRegionInfo hri = new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z")); MetaTableAccessor.addRegionToMeta(meta, hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 846f8e6ff4d..6307c4c0e5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -253,7 +253,7 @@ public class TestMasterOperationsForRegionReplicas { ADMIN.disableTable(table); // now delete one replica info from all the rows // this is to make the meta appear to be only partially updated - Table metaTable = new HTable(TableName.META_TABLE_NAME, ADMIN.getConnection()); + Table metaTable = ADMIN.getConnection().getTable(TableName.META_TABLE_NAME); for (byte[] row : tableRows) { Delete deleteOneReplicaLocation = new Delete(row); deleteOneReplicaLocation.deleteColumns(HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java index 56961d5a2a9..1594f807c19 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java @@ -99,7 +99,7 @@ public class TestMasterRestartAfterDisablingTable { TableState.State.DISABLING)); log("Enabling table\n"); // Need a new Admin, the previous one is on the old master - Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + Admin admin = TEST_UTIL.getHBaseAdmin(); admin.enableTable(table); admin.close(); log("Waiting for no more RIT\n"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 374366ef1e6..7ee69f91bdb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -65,7 +65,7 @@ public class TestMasterTransitions { TEST_UTIL.startMiniCluster(2); // Create a table of three families. This will assign a region. TEST_UTIL.createTable(TABLENAME, FAMILIES); - HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); + HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME); int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily()); TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME); addToEachStartKey(countOfRegions); @@ -480,9 +480,8 @@ public class TestMasterTransitions { * @throws IOException */ private static int addToEachStartKey(final int expected) throws IOException { - Table t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); - Table meta = new HTable(TEST_UTIL.getConfiguration(), - TableName.META_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(TABLENAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); int rows = 0; Scan scan = new Scan(); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java index ad22fe9cd51..b51f7c710f4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java @@ -35,8 +35,6 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -66,8 +64,6 @@ public class TestRestartCluster { @Test (timeout=300000) public void testClusterRestart() throws Exception { UTIL.startMiniCluster(3); - Connection connection = UTIL.getConnection(); - while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) { Threads.sleep(1); } @@ -80,7 +76,7 @@ public class TestRestartCluster { } List allRegions = - MetaScanner.listAllRegions(UTIL.getConfiguration(), connection, true); + MetaScanner.listAllRegions(UTIL.getConfiguration(), UTIL.getConnection(), true); assertEquals(4, allRegions.size()); LOG.info("\n\nShutting down cluster"); @@ -95,8 +91,8 @@ public class TestRestartCluster { // Need to use a new 'Configuration' so we make a new HConnection. // Otherwise we're reusing an HConnection that has gone stale because // the shutdown of the cluster also called shut of the connection. - allRegions = - MetaScanner.listAllRegions(new Configuration(UTIL.getConfiguration()), connection, true); + allRegions = MetaScanner + .listAllRegions(new Configuration(UTIL.getConfiguration()), UTIL.getConnection(), true); assertEquals(4, allRegions.size()); LOG.info("\n\nWaiting for tables to be available"); for(TableName TABLE: TABLES) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java index 9a72e77baaa..a2f08abd753 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java @@ -290,10 +290,10 @@ public class TestSnapshotFromMaster { htd.setCompactionEnabled(false); UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration()); // load the table (creates 4 hfiles) - UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM); + UTIL.loadTable(UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM); UTIL.flush(TABLE_NAME); // Put some more data into the table so for sure we get more storefiles. - UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM); + UTIL.loadTable((HTable) UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM); // disable the table so we can take a snapshot admin.disableTable(TABLE_NAME); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java index ce6abda9669..5b2f4f6fcf7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Bytes; @@ -69,7 +70,7 @@ public class TestTableDeleteFamilyHandler { // Create a table of three families. This will assign a region. TEST_UTIL.createTable(TABLENAME, FAMILIES); - HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); + Table t = TEST_UTIL.getConnection().getTable(TABLENAME); while(TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager() .getRegionStates().getRegionsInTransition().size() > 0) { Thread.sleep(100); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java index 3ba01bcb36d..3f5f9058be7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionState.java @@ -89,7 +89,7 @@ public class TestCompactionState { Table ht = null; try { ht = TEST_UTIL.createTable(table, family); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); try { admin.compact(table, fakecf); } catch (IOException ioe) { @@ -137,7 +137,7 @@ public class TestCompactionState { int countBefore = countStoreFilesInFamilies(regions, families); int countBeforeSingleFamily = countStoreFilesInFamily(regions, family); assertTrue(countBefore > 0); // there should be some data files - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); if (expectedState == CompactionState.MINOR) { if (singleFamily) { admin.compact(table.getName(), family); @@ -154,10 +154,10 @@ public class TestCompactionState { long curt = System.currentTimeMillis(); long waitTime = 5000; long endt = curt + waitTime; - CompactionState state = admin.getCompactionState(table.getName()); + CompactionState state = admin.getCompactionState(table); while (state == CompactionState.NONE && curt < endt) { Thread.sleep(10); - state = admin.getCompactionState(table.getName()); + state = admin.getCompactionState(table); curt = System.currentTimeMillis(); } // Now, should have the right compaction state, @@ -169,10 +169,10 @@ public class TestCompactionState { } } else { // Wait until the compaction is done - state = admin.getCompactionState(table.getName()); + state = admin.getCompactionState(table); while (state != CompactionState.NONE && curt < endt) { Thread.sleep(10); - state = admin.getCompactionState(table.getName()); + state = admin.getCompactionState(table); } // Now, compaction should be done. assertEquals(CompactionState.NONE, state); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index 941f6d2e916..af86b4e0621 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -208,7 +208,7 @@ public class TestEncryptionKeyRotation { TEST_UTIL.getHBaseAdmin().createTable(htd); TEST_UTIL.waitTableAvailable(htd.getName(), 5000); // Create a store file - Table table = new HTable(conf, htd.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); try { table.put(new Put(Bytes.toBytes("testrow")) .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index efae472b24f..29a58a65ee7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -103,7 +103,7 @@ public class TestEncryptionRandomKeying { TEST_UTIL.waitTableAvailable(htd.getName(), 5000); // Create a store file - Table table = new HTable(conf, htd.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); try { table.put(new Put(Bytes.toBytes("testrow")) .add(hcd.getName(), Bytes.toBytes("q"), Bytes.toBytes("value"))); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 8b7edeb3cd2..4a6ba4141c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -331,7 +331,7 @@ public class TestEndToEndSplitTransaction { HTable table = null; try { //HTable.getStartEndKeys() - table = new HTable(conf, tableName); + table = (HTable) connection.getTable(tableName); Pair keys = table.getStartEndKeys(); verifyStartEndKeys(keys); @@ -433,7 +433,8 @@ public class TestEndToEndSplitTransaction { long start = System.currentTimeMillis(); log("blocking until region is split:" + Bytes.toStringBinary(regionName)); HRegionInfo daughterA = null, daughterB = null; - Table metaTable = new HTable(conf, TableName.META_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table metaTable = connection.getTable(TableName.META_TABLE_NAME); try { Result result = null; @@ -476,6 +477,7 @@ public class TestEndToEndSplitTransaction { } } finally { IOUtils.closeQuietly(metaTable); + IOUtils.closeQuietly(connection); } } @@ -505,7 +507,8 @@ public class TestEndToEndSplitTransaction { throws IOException, InterruptedException { log("blocking until region is opened for reading:" + hri.getRegionNameAsString()); long start = System.currentTimeMillis(); - Table table = new HTable(conf, hri.getTable()); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(hri.getTable()); try { byte [] row = hri.getStartKey(); @@ -523,7 +526,7 @@ public class TestEndToEndSplitTransaction { } } finally { IOUtils.closeQuietly(table); + IOUtils.closeQuietly(connection); } } } - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index 9a2cc82ab83..c9608c9d2ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; @@ -190,7 +191,7 @@ public class TestFSErrorsExposed { TableName tableName = TableName.valueOf("table"); byte[] fam = Bytes.toBytes("fam"); - Admin admin = new HBaseAdmin(util.getConfiguration()); + Admin admin = util.getHBaseAdmin(); HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(new HColumnDescriptor(fam) .setMaxVersions(1) @@ -201,7 +202,7 @@ public class TestFSErrorsExposed { util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Make a new Configuration so it makes a new connection that has the // above configuration on it; else we use the old one w/ 10 as default. - HTable table = new HTable(new Configuration(util.getConfiguration()), tableName); + Table table = util.getConnection().getTable(tableName); // Load some data util.loadTable(table, fam, false); 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 ce2869b8fbb..9b8dabf69f6 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 @@ -76,7 +76,7 @@ public class TestHRegionOnCluster { // Put data: r1->v1 Log.info("Loading r1 to v1 into " + TABLENAME); - HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLENAME); + HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME); putDataAndVerify(table, "r1", FAMILY, "v1", 1); TEST_UTIL.waitUntilAllRegionsAssigned(table.getName()); 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 0e94e68cfb2..d2bfb52f04c 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 @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.RpcRetryingCaller; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -206,7 +207,7 @@ public class TestHRegionServerBulkLoad { */ public static class AtomicScanReader extends RepeatingTestThread { byte targetFamilies[][]; - HTable table; + Table table; AtomicLong numScans = new AtomicLong(); AtomicLong numRowsScanned = new AtomicLong(); TableName TABLE_NAME; @@ -216,7 +217,7 @@ public class TestHRegionServerBulkLoad { super(ctx); this.TABLE_NAME = TABLE_NAME; this.targetFamilies = targetFamilies; - table = new HTable(conf, TABLE_NAME); + table = UTIL.getConnection().getTable(TABLE_NAME); } public void doAnAction() throws Exception { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java index b8e63824365..d30b5f8c086 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java @@ -101,7 +101,7 @@ public class TestJoinedScanners { desc.addFamily(hcd); } htu.getHBaseAdmin().createTable(desc); - Table ht = new HTable(htu.getConfiguration(), tableName); + Table ht = htu.getConnection().getTable(tableName); long rows_to_insert = 1000; int insert_batch = 20; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index cbe79fe5552..538e47e60c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -133,7 +133,7 @@ public class TestRegionReplicas { openRegion(HTU, getRS(), hriSecondary); Table meta = null; try { - meta = new HTable(HTU.getConfiguration(), TableName.META_TABLE_NAME); + meta = HTU.getConnection().getTable(TableName.META_TABLE_NAME); TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName() , getRS().getServerName(), -1, 1, false); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index 4471f761295..b95b20a21c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -109,10 +109,10 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tName, cfName); - new HTable(conf, tName).close(); //wait for the table to come up. + TEST_UTIL.getConnection().getTable(tName).close(); //wait for the table to come up. // Do a first put to be sure that the connection is established, meta is there and so on. - HTable table = new HTable(conf, tName); + HTable table = (HTable) TEST_UTIL.getConnection().getTable(tName); Put p = new Put(row); p.add(cfName, qualifier, initValue); table.put(p); @@ -189,9 +189,7 @@ public class TestRegionServerMetrics { metricsRegionServer.getRegionServerWrapper().forceRecompute(); - TEST_UTIL.createTable(tableName, cf); - - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.createTable(tableName, cf); Put p = new Put(row); p.add(cf, qualifier, val); @@ -219,10 +217,8 @@ public class TestRegionServerMetrics { long stores = metricsHelper.getGaugeLong("storeCount", serverSource); long storeFiles = metricsHelper.getGaugeLong("storeFileCount", serverSource); - TEST_UTIL.createTable(tableName, cf); - //Force a hfile. - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.createTable(tableName, cf); Put p = new Put(row); p.add(cf, qualifier, val); t.put(p); @@ -246,8 +242,7 @@ public class TestRegionServerMetrics { byte[] valTwo = Bytes.toBytes("ValueTwo"); byte[] valThree = Bytes.toBytes("ValueThree"); - TEST_UTIL.createTable(tableName, cf); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.createTable(tableName, cf); Put p = new Put(row); p.add(cf, qualifier, valOne); t.put(p); @@ -277,9 +272,7 @@ public class TestRegionServerMetrics { byte[] val = Bytes.toBytes(0l); - TEST_UTIL.createTable(tableName, cf); - Table t = new HTable(conf, tableName); - + Table t = TEST_UTIL.createTable(tableName, cf); Put p = new Put(row); p.add(cf, qualifier, val); t.put(p); @@ -306,9 +299,7 @@ public class TestRegionServerMetrics { byte[] val = Bytes.toBytes("One"); - TEST_UTIL.createTable(tableName, cf); - Table t = new HTable(conf, tableName); - + Table t = TEST_UTIL.createTable(tableName, cf); Put p = new Put(row); p.add(cf, qualifier, val); t.put(p); @@ -334,8 +325,7 @@ public class TestRegionServerMetrics { byte[] val = Bytes.toBytes("One"); - TEST_UTIL.createTable(tableName, cf); - HTable t = new HTable(conf, tableName); + HTable t = TEST_UTIL.createTable(tableName, cf); t.setAutoFlushTo(false); for (int insertCount =0; insertCount < 100; insertCount++) { Put p = new Put(Bytes.toBytes("" + insertCount + "row")); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java index 309efbfff7f..c6c9f505f2c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSCVFWithMiniCluster.java @@ -83,7 +83,7 @@ public class TestSCVFWithMiniCluster { destroy(admin, HBASE_TABLE_NAME); create(admin, HBASE_TABLE_NAME, FAMILY_A, FAMILY_B); admin.close(); - htable = new HTable(util.getConfiguration(), HBASE_TABLE_NAME); + htable = util.getConnection().getTable(HBASE_TABLE_NAME); /* Add some values */ List puts = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index 49ded212120..bd5439af83b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableNotFoundException; @@ -72,17 +73,17 @@ public class TestScannerWithBulkload { public void testBulkLoad() throws Exception { TableName tableName = TableName.valueOf("testBulkLoad"); long l = System.currentTimeMillis(); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); createTable(admin, tableName); Scan scan = createScan(); - final HTable table = init(admin, l, scan, tableName); + final Table table = init(admin, l, scan, tableName); // use bulkload final Path hfilePath = writeToHFile(l, "/temp/testBulkLoad/", "/temp/testBulkLoad/col/file", false); Configuration conf = TEST_UTIL.getConfiguration(); conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true); final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf); - bulkload.doBulkLoad(hfilePath, table); + bulkload.doBulkLoad(hfilePath, (HTable) table); ResultScanner scanner = table.getScanner(scan); Result result = scanner.next(); result = scanAfterBulkLoad(scanner, result, "version2"); @@ -165,8 +166,8 @@ public class TestScannerWithBulkload { return hfilePath; } - private HTable init(HBaseAdmin admin, long l, Scan scan, TableName tableName) throws Exception { - HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName); + private Table init(HBaseAdmin admin, long l, Scan scan, TableName tableName) throws Exception { + Table table = TEST_UTIL.getConnection().getTable(tableName); Put put0 = new Put(Bytes.toBytes("row1")); put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes .toBytes("version0"))); @@ -200,10 +201,10 @@ public class TestScannerWithBulkload { public void testBulkLoadWithParallelScan() throws Exception { TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan"); final long l = System.currentTimeMillis(); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); createTable(admin, tableName); Scan scan = createScan(); - final HTable table = init(admin, l, scan, tableName); + final Table table = init(admin, l, scan, tableName); // use bulkload final Path hfilePath = writeToHFile(l, "/temp/testBulkLoadWithParallelScan/", "/temp/testBulkLoadWithParallelScan/col/file", false); @@ -221,7 +222,7 @@ public class TestScannerWithBulkload { Bytes.toBytes("version0"))); table.put(put1); table.flushCommits(); - bulkload.doBulkLoad(hfilePath, table); + bulkload.doBulkLoad(hfilePath, (HTable) table); latch.countDown(); } catch (TableNotFoundException e) { } catch (IOException e) { @@ -242,17 +243,17 @@ public class TestScannerWithBulkload { public void testBulkLoadNativeHFile() throws Exception { TableName tableName = TableName.valueOf("testBulkLoadNativeHFile"); long l = System.currentTimeMillis(); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); createTable(admin, tableName); Scan scan = createScan(); - final HTable table = init(admin, l, scan, tableName); + final Table table = init(admin, l, scan, tableName); // use bulkload final Path hfilePath = writeToHFile(l, "/temp/testBulkLoadNativeHFile/", "/temp/testBulkLoadNativeHFile/col/file", true); Configuration conf = TEST_UTIL.getConfiguration(); conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true); final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf); - bulkload.doBulkLoad(hfilePath, table); + bulkload.doBulkLoad(hfilePath, (HTable) table); ResultScanner scanner = table.getScanner(scan); Result result = scanner.next(); // We had 'version0', 'version1' for 'row1,col:q' in the table. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java index c6c3cb75e5c..f5cf33d3998 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java @@ -185,7 +185,7 @@ public class TestServerCustomProtocol { @Test public void testSingleProxy() throws Throwable { - Table table = new HTable(util.getConfiguration(), TEST_TABLE); + Table table = util.getConnection().getTable(TEST_TABLE); Map results = ping(table, null, null); // There are three regions so should get back three results. assertEquals(3, results.size()); @@ -308,7 +308,7 @@ public class TestServerCustomProtocol { @Test public void testSingleMethod() throws Throwable { - try (HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); Map results = table.coprocessorService(PingProtos.PingService.class, null, ROW_A, @@ -337,7 +337,7 @@ public class TestServerCustomProtocol { @Test public void testRowRange() throws Throwable { - try (HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); for (Entry e: table.getRegionLocations().entrySet()) { LOG.info("Region " + e.getKey().getRegionNameAsString() + ", servername=" + e.getValue()); @@ -417,7 +417,7 @@ public class TestServerCustomProtocol { @Test public void testCompoundCall() throws Throwable { - try (HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); Map results = compoundOfHelloAndPing(table, ROW_A, ROW_C); verifyRegionResults(locator, results, "Hello, pong", ROW_A); @@ -428,7 +428,7 @@ public class TestServerCustomProtocol { @Test public void testNullCall() throws Throwable { - try(HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try(HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); Map results = hello(table, null, ROW_A, ROW_C); verifyRegionResults(locator, results, "Who are you?", ROW_A); @@ -439,7 +439,7 @@ public class TestServerCustomProtocol { @Test public void testNullReturn() throws Throwable { - try (HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { RegionLocator locator = table.getRegionLocator(); Map results = hello(table, "nobody", ROW_A, ROW_C); verifyRegionResults(locator, results, null, ROW_A); @@ -450,7 +450,7 @@ public class TestServerCustomProtocol { @Test public void testEmptyReturnType() throws Throwable { - try (HTable table = new HTable(util.getConfiguration(), TEST_TABLE)) { + try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) { Map results = noop(table, ROW_A, ROW_C); assertEquals("Should have results from three regions", 3, results.size()); // all results should be null diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 0d574f6845c..bd66b39e90c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -137,7 +137,7 @@ public class TestSplitTransactionOnCluster { @Before public void setup() throws IOException { TESTING_UTIL.ensureSomeNonStoppedRegionServersAvailable(NB_SERVERS); - this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration()); + this.admin = TESTING_UTIL.getHBaseAdmin(); this.cluster = TESTING_UTIL.getMiniHBaseCluster(); } @@ -224,8 +224,6 @@ public class TestSplitTransactionOnCluster { @Test(timeout = 60000) public void testSplitFailedCompactionAndSplit() throws Exception { final TableName tableName = TableName.valueOf("testSplitFailedCompactionAndSplit"); - Configuration conf = TESTING_UTIL.getConfiguration(); - HBaseAdmin admin = new HBaseAdmin(conf); // Create table then get the single region for our new table. HTableDescriptor htd = new HTableDescriptor(tableName); byte[] cf = Bytes.toBytes("cf"); @@ -242,7 +240,7 @@ public class TestSplitTransactionOnCluster { int regionServerIndex = cluster.getServerWith(region.getRegionName()); HRegionServer regionServer = cluster.getRegionServer(regionServerIndex); - Table t = new HTable(conf, tableName); + Table t = TESTING_UTIL.getConnection().getTable(tableName); // insert data insertData(tableName, admin, t); insertData(tableName, admin, t); @@ -480,7 +478,7 @@ public class TestSplitTransactionOnCluster { HColumnDescriptor hcd = new HColumnDescriptor("col"); htd.addFamily(hcd); admin.createTable(htd); - Table table = new HTable(conf, userTableName); + Table table = TESTING_UTIL.getConnection().getTable(userTableName); try { for (int i = 0; i <= 5; i++) { String row = "row" + i; @@ -590,7 +588,7 @@ public class TestSplitTransactionOnCluster { HMaster master = abortAndWaitForMaster(); - this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration()); + this.admin = TESTING_UTIL.getHBaseAdmin(); // Update the region to be offline and split, so that HRegionInfo#equals // returns true in checking rebuilt region states map. @@ -927,8 +925,8 @@ public class TestSplitTransactionOnCluster { Table table1 = null; Table table2 = null; try { - table1 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); - table2 = new HTable(TESTING_UTIL.getConfiguration(), firstTable); + table1 = TESTING_UTIL.getConnection().getTable(firstTable); + table2 = TESTING_UTIL.getConnection().getTable(firstTable); insertData(firstTable, admin, table1); insertData(secondTable, admin, table2); admin.split(firstTable, "row2".getBytes()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java index eaea83e1fc7..14c6ca9e337 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java @@ -119,7 +119,7 @@ public class TestTags { Admin admin = TEST_UTIL.getHBaseAdmin(); admin.createTable(desc); byte[] value = Bytes.toBytes("value"); - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(row); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, value); put.setAttribute("visibility", Bytes.toBytes("myTag")); @@ -185,7 +185,7 @@ public class TestTags { Admin admin = TEST_UTIL.getHBaseAdmin(); admin.createTable(desc); - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(row); byte[] value = Bytes.toBytes("value"); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, value); @@ -275,7 +275,7 @@ public class TestTags { Admin admin = TEST_UTIL.getHBaseAdmin(); admin.createTable(desc); try { - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(row); byte[] value = Bytes.toBytes("value"); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, value); @@ -388,7 +388,7 @@ public class TestTags { Table table = null; try { - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(row1); byte[] v = Bytes.toBytes(2L); put.add(f, q, v); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index b4cb213e906..4e07040b970 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -128,8 +128,7 @@ public class TestLogRollAbort { LOG.info("Starting testRSAbortWithUnflushedEdits()"); // When the hbase:meta table can be opened, the region servers are running - new HTable(TEST_UTIL.getConfiguration(), - TableName.META_TABLE_NAME).close(); + TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); // Create the test table and open it TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); @@ -137,9 +136,8 @@ public class TestLogRollAbort { desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); try { - HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); WAL log = server.getWAL(null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java index cdbdf6fddd3..5216ee57fd7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollPeriod.java @@ -77,7 +77,7 @@ public class TestLogRollPeriod { TableName tableName = TableName.valueOf("TestLogRollPeriodNoEdits"); TEST_UTIL.createTable(tableName, "cf"); try { - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); WAL log = server.getWAL(null); @@ -102,7 +102,7 @@ public class TestLogRollPeriod { try { HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName); WAL log = server.getWAL(null); - final Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + final Table table = TEST_UTIL.getConnection().getTable(tableName); Thread writerThread = new Thread("writer") { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 86e77ad2b51..1da3662def5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -172,7 +172,7 @@ public class TestLogRolling { private void startAndWriteData() throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running - new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); + TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); this.server = cluster.getRegionServerThreads().get(0).getRegionServer(); Table table = createTestTable(this.tableName); @@ -311,7 +311,7 @@ public class TestLogRolling { desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); assertTrue(table.isAutoFlush()); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); @@ -420,7 +420,7 @@ public class TestLogRolling { LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); // When the hbase:meta table can be opened, the region servers are running - Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); try { this.server = cluster.getRegionServer(0); @@ -429,7 +429,7 @@ public class TestLogRolling { desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(desc.getTableName()); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); final WAL log = server.getWAL(null); @@ -569,7 +569,7 @@ public class TestLogRolling { Table table2 = null; // When the hbase:meta table can be opened, the region servers are running - Table t = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); try { table = createTestTable(getName()); table2 = createTestTable(getName() + "1"); @@ -635,7 +635,7 @@ public class TestLogRolling { HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); admin.createTable(desc); - return new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); + return TEST_UTIL.getConnection().getTable(desc.getTableName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 6cdfe3b4157..c441969b35b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -186,7 +186,7 @@ public class TestWALReplay { byte[] value = Bytes.toBytes("testV"); byte[][] familys = { family1, family2 }; TEST_UTIL.createTable(tableName, familys); - Table htable = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table htable = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(Bytes.toBytes("r1")); put.add(family1, qualifier, value); htable.put(put); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index f9aeb6f5cfb..b30820b385e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -131,14 +131,14 @@ public class TestMultiSlaveReplication { utility3.startMiniCluster(); ReplicationAdmin admin1 = new ReplicationAdmin(conf1); - new HBaseAdmin(conf1).createTable(table); - new HBaseAdmin(conf2).createTable(table); - new HBaseAdmin(conf3).createTable(table); - Table htable1 = new HTable(conf1, tableName); + utility1.getHBaseAdmin().createTable(table); + utility2.getHBaseAdmin().createTable(table); + utility3.getHBaseAdmin().createTable(table); + Table htable1 = utility1.getConnection().getTable(tableName); htable1.setWriteBufferSize(1024); - Table htable2 = new HTable(conf2, tableName); + Table htable2 = utility2.getConnection().getTable(tableName); htable2.setWriteBufferSize(1024); - Table htable3 = new HTable(conf3, tableName); + Table htable3 = utility3.getConnection().getTable(tableName); htable3.setWriteBufferSize(1024); admin1.addPeer("1", utility2.getClusterKey()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index 67f2031f28a..7ca9fed3809 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -119,7 +119,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas Put put = new Put(row); put.add(famName, row, row); - htable1 = new HTable(conf1, tableName); + htable1 = utility1.getConnection().getTable(tableName); htable1.put(put); Get get = new Get(row); @@ -136,7 +136,5 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas break; } } - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 9a3b581c65d..bfb01dbf6e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -116,7 +116,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { final byte[] v1 = Bytes.toBytes("v1"); final byte[] v2 = Bytes.toBytes("v2"); final byte[] v3 = Bytes.toBytes("v3"); - htable1 = new HTable(conf1, tableName); + htable1 = utility1.getConnection().getTable(tableName); long t = EnvironmentEdgeManager.currentTime(); // create three versions for "row" @@ -203,7 +203,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { Put put = new Put(row); put.add(famName, row, row); - htable1 = new HTable(conf1, tableName); + htable1 = utility1.getConnection().getTable(tableName); htable1.put(put); Get get = new Get(row); @@ -519,7 +519,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { final String colFam = "cf1"; final int numOfTables = 3; - HBaseAdmin hadmin = new HBaseAdmin(conf1); + HBaseAdmin hadmin = utility1.getHBaseAdmin(); // Create Tables for (int i = 0; i < numOfTables; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java index 58eb19f8887..26acdabda38 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java @@ -17,28 +17,32 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertEquals; +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.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +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.Delete; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; 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.*; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HBaseAdmin; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.ReplicationTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; +import static org.junit.Assert.assertEquals; @Category({ReplicationTests.class, LargeTests.class}) public class TestReplicationSyncUpTool extends TestReplicationBase { @@ -179,26 +183,29 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { ReplicationAdmin admin1 = new ReplicationAdmin(conf1); ReplicationAdmin admin2 = new ReplicationAdmin(conf2); - Admin ha = new HBaseAdmin(conf1); + Admin ha = utility1.getHBaseAdmin(); ha.createTable(t1_syncupSource); ha.createTable(t2_syncupSource); ha.close(); - ha = new HBaseAdmin(conf2); + ha = utility2.getHBaseAdmin(); ha.createTable(t1_syncupTarget); ha.createTable(t2_syncupTarget); ha.close(); + Connection connection1 = ConnectionFactory.createConnection(utility1.getConfiguration()); + Connection connection2 = ConnectionFactory.createConnection(utility2.getConfiguration()); + // Get HTable from Master - ht1Source = new HTable(conf1, t1_su); + ht1Source = connection1.getTable(t1_su); ht1Source.setWriteBufferSize(1024); - ht2Source = new HTable(conf1, t2_su); + ht2Source = connection1.getTable(t2_su); ht1Source.setWriteBufferSize(1024); // Get HTable from Peer1 - ht1TargetAtPeer1 = new HTable(conf2, t1_su); + ht1TargetAtPeer1 = connection2.getTable(t1_su); ht1TargetAtPeer1.setWriteBufferSize(1024); - ht2TargetAtPeer1 = new HTable(conf2, t2_su); + ht2TargetAtPeer1 = connection2.getTable(t2_su); ht2TargetAtPeer1.setWriteBufferSize(1024); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index fc06d150358..adf3c0eb366 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -150,9 +150,9 @@ public class TestReplicationWithTags { Admin admin = conn.getAdmin()) { admin.createTable(table, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); } - htable1 = new HTable(conf1, TABLE_NAME); + htable1 = utility1.getConnection().getTable(TABLE_NAME); htable1.setWriteBufferSize(1024); - htable2 = new HTable(conf2, TABLE_NAME); + htable2 = utility2.getConnection().getTable(TABLE_NAME); } /** @@ -171,7 +171,7 @@ public class TestReplicationWithTags { put.setAttribute("visibility", Bytes.toBytes("myTag3")); put.add(FAMILY, ROW, ROW); - htable1 = new HTable(conf1, TABLE_NAME); + htable1 = utility1.getConnection().getTable(TABLE_NAME); htable1.put(put); Get get = new Get(ROW); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java index 2eec09d0c65..d6a6f03f47c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java @@ -29,6 +29,8 @@ import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.testclassification.SecurityTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.TableName; @@ -123,7 +125,8 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TABLE); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; @@ -139,6 +142,7 @@ public class TestAccessControlFilter extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }); @@ -149,7 +153,8 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TABLE); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; @@ -164,6 +169,7 @@ public class TestAccessControlFilter extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }); @@ -174,7 +180,8 @@ public class TestAccessControlFilter extends SecureTestUtil { Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TABLE); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TABLE); try { ResultScanner rs = t.getScanner(new Scan()); int rowcnt = 0; @@ -189,6 +196,7 @@ public class TestAccessControlFilter extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 27ee91580e7..b8f41c4d56d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -705,7 +705,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(TEST_FAMILY); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.get(g); } finally { @@ -723,7 +723,7 @@ public class TestAccessController extends SecureTestUtil { Scan s = new Scan(); s.addFamily(TEST_FAMILY); - Table table = new HTable(conf, TEST_TABLE.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { ResultScanner scanner = table.getScanner(s); try { @@ -752,7 +752,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1)); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.put(p); } finally { @@ -769,7 +769,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(TEST_FAMILY); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.delete(d); } finally { @@ -786,7 +786,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Increment inc = new Increment(TEST_ROW); inc.addColumn(TEST_FAMILY, TEST_QUALIFIER, 1); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.increment(inc); } finally { @@ -806,7 +806,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(TEST_FAMILY); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.checkAndDelete(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("test_value"), d); @@ -824,7 +824,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1)); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.checkAndPut(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("test_value"), p); @@ -960,7 +960,7 @@ public class TestAccessController extends SecureTestUtil { put.add(TEST_FAMILY, qualifier, Bytes.toBytes(1)); Append append = new Append(row); append.add(TEST_FAMILY, qualifier, Bytes.toBytes(2)); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { t.put(put); t.append(append); @@ -980,7 +980,8 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction grantAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -989,6 +990,7 @@ public class TestAccessController extends SecureTestUtil { TEST_FAMILY, null, Action.READ); } finally { acl.close(); + connection.close(); } return null; } @@ -997,7 +999,8 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction revokeAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -1006,6 +1009,7 @@ public class TestAccessController extends SecureTestUtil { TEST_FAMILY, null, Action.READ); } finally { acl.close(); + connection.close(); } return null; } @@ -1014,7 +1018,8 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getTablePermissionsAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getTableName().getName()); AccessControlService.BlockingInterface protocol = @@ -1022,6 +1027,7 @@ public class TestAccessController extends SecureTestUtil { ProtobufUtil.getUserPermissions(protocol, TEST_TABLE.getTableName()); } finally { acl.close(); + connection.close(); } return null; } @@ -1030,7 +1036,8 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getGlobalPermissionsAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -1038,6 +1045,7 @@ public class TestAccessController extends SecureTestUtil { ProtobufUtil.getUserPermissions(protocol); } finally { acl.close(); + connection.close(); } return null; } @@ -1090,7 +1098,7 @@ public class TestAccessController extends SecureTestUtil { Put p = new Put(Bytes.toBytes("a")); p.add(family1, qualifier, Bytes.toBytes("v1")); p.add(family2, qualifier, Bytes.toBytes("v2")); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.put(p); } finally { @@ -1105,7 +1113,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(Bytes.toBytes("a")); p.add(family1, qualifier, Bytes.toBytes("v1")); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.put(p); } finally { @@ -1120,7 +1128,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(Bytes.toBytes("a")); p.add(family2, qualifier, Bytes.toBytes("v2")); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.put(p); } finally { @@ -1136,7 +1144,7 @@ public class TestAccessController extends SecureTestUtil { Get g = new Get(TEST_ROW); g.addFamily(family1); g.addFamily(family2); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.get(g); } finally { @@ -1151,7 +1159,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(family1); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.get(g); } finally { @@ -1166,7 +1174,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addFamily(family2); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.get(g); } finally { @@ -1182,7 +1190,7 @@ public class TestAccessController extends SecureTestUtil { Delete d = new Delete(TEST_ROW); d.deleteFamily(family1); d.deleteFamily(family2); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.delete(d); } finally { @@ -1197,7 +1205,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(family1); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.delete(d); } finally { @@ -1212,7 +1220,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Delete d = new Delete(TEST_ROW); d.deleteFamily(family2); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.delete(d); } finally { @@ -1360,7 +1368,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Get g = new Get(TEST_ROW); g.addColumn(family1, qualifier); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.get(g); } finally { @@ -1375,7 +1383,7 @@ public class TestAccessController extends SecureTestUtil { public Object run() throws Exception { Put p = new Put(TEST_ROW); p.add(family1, qualifier, Bytes.toBytes("v1")); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.put(p); } finally { @@ -1391,7 +1399,7 @@ public class TestAccessController extends SecureTestUtil { Delete d = new Delete(TEST_ROW); d.deleteColumn(family1, qualifier); // d.deleteFamily(family1); - Table t = new HTable(conf, tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); try { t.delete(d); } finally { @@ -1470,7 +1478,7 @@ public class TestAccessController extends SecureTestUtil { List perms; - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1497,7 +1505,7 @@ public class TestAccessController extends SecureTestUtil { grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier, Permission.Action.READ); - acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1522,7 +1530,7 @@ public class TestAccessController extends SecureTestUtil { tableName, family1, qualifier, Permission.Action.WRITE, Permission.Action.READ); - acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1541,7 +1549,7 @@ public class TestAccessController extends SecureTestUtil { revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier, Permission.Action.WRITE, Permission.Action.READ); - acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1561,7 +1569,7 @@ public class TestAccessController extends SecureTestUtil { htd.setOwner(newOwner); admin.modifyTable(tableName, htd); - acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(tableName.getName()); AccessControlService.BlockingInterface protocol = @@ -1583,7 +1591,7 @@ public class TestAccessController extends SecureTestUtil { @Test public void testGlobalPermissionList() throws Exception { List perms; - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -1618,7 +1626,7 @@ public class TestAccessController extends SecureTestUtil { AccessControlProtos.GlobalPermission.newBuilder() .addAction(ProtobufUtil.toPermissionAction(a)).build())); } - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); AccessControlService.BlockingInterface protocol = @@ -1648,7 +1656,7 @@ public class TestAccessController extends SecureTestUtil { for (Permission p : perms) { request.addPermission(ProtobufUtil.toPermission(p)); } - Table acl = new HTable(conf, table); + Table acl = TEST_UTIL.getConnection().getTable(table); try { AccessControlService.BlockingInterface protocol = AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0])); @@ -1798,7 +1806,7 @@ public class TestAccessController extends SecureTestUtil { .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE.getTableName())) .addAction(AccessControlProtos.Permission.Action.CREATE)) ).build(); - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); AccessControlService.BlockingInterface protocol = @@ -2160,7 +2168,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW)); } finally { @@ -2187,7 +2195,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW)); } finally { @@ -2229,7 +2237,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW)); } finally { @@ -2270,7 +2278,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction getAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW)); } finally { @@ -2375,7 +2383,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction execEndpointAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { BlockingRpcChannel service = t.coprocessorService(HConstants.EMPTY_BYTE_ARRAY); PingCoprocessor.newBlockingStub(service).noop(null, NoopRequest.newBuilder().build()); @@ -2404,7 +2412,7 @@ public class TestAccessController extends SecureTestUtil { AccessTestAction putWithReservedTag = new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER, HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java index 2bde357198d..f6066ad0452 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController2.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.Permission.Action; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -164,7 +165,7 @@ public class TestAccessController2 extends SecureTestUtil { AccessTestAction writeAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { t.put(new Put(TEST_ROW).add(AccessControlLists.ACL_LIST_FAMILY, TEST_QUALIFIER, TEST_VALUE)); @@ -187,7 +188,7 @@ public class TestAccessController2 extends SecureTestUtil { AccessTestAction scanAction = new AccessTestAction() { @Override public Object run() throws Exception { - HTable t = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { ResultScanner s = t.getScanner(new Scan()); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java index 3a8d66224d8..e239647d466 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLWithMultipleVersions.java @@ -154,7 +154,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Put p; // with ro ACL @@ -176,6 +177,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { t.put(p); } finally { t.close(); + connection.close(); } return null; } @@ -188,11 +190,13 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { public Object run() throws Exception { Get get = new Get(TEST_ROW); get.setMaxVersions(10); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { t.close(); + connection.close(); } } }; @@ -202,11 +206,13 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { public Object run() throws Exception { Get get = new Get(TEST_ROW); get.setMaxVersions(10); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { t.close(); + connection.close(); } } }; @@ -219,7 +225,8 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO); @@ -233,6 +240,7 @@ public class TestCellACLWithMultipleVersions extends SecureTestUtil { t.put(p); } finally { t.close(); + connection.close(); } return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java index 4bc819e65ae..b7cbc5266a5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCellACLs.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNotFoundException; 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.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; @@ -145,7 +147,8 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Put p; // with ro ACL @@ -163,6 +166,7 @@ public class TestCellACLs extends SecureTestUtil { t.put(p); } finally { t.close(); + connection.close(); } return null; } @@ -174,11 +178,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { t.close(); + connection.close(); } } }; @@ -187,11 +193,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { t.close(); + connection.close(); } } }; @@ -200,11 +208,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { t.close(); + connection.close(); } } }; @@ -213,7 +223,8 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q4); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { return t.get(get).listCells(); } finally { @@ -245,7 +256,8 @@ public class TestCellACLs extends SecureTestUtil { scan.setStartRow(TEST_ROW); scan.setStopRow(Bytes.add(TEST_ROW, new byte[]{ 0 } )); scan.addFamily(TEST_FAMILY); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { ResultScanner scanner = t.getScanner(scan); Result result = null; @@ -257,6 +269,7 @@ public class TestCellACLs extends SecureTestUtil { } while (result != null); } finally { t.close(); + connection.close(); } return scanResults; } @@ -278,11 +291,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, 1L); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.increment(i); } finally { t.close(); + connection.close(); } return null; } @@ -292,11 +307,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.increment(i); } finally { t.close(); + connection.close(); } return null; } @@ -308,11 +325,13 @@ public class TestCellACLs extends SecureTestUtil { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L); // Tag this increment with an ACL that denies write permissions to USER_OTHER i.setACL(USER_OTHER.getShortName(), new Permission(Action.READ)); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.increment(i); } finally { t.close(); + connection.close(); } return null; } @@ -322,11 +341,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3, 1L); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.increment(i); } finally { t.close(); + connection.close(); } return null; } @@ -349,11 +370,13 @@ public class TestCellACLs extends SecureTestUtil { @Override public Object run() throws Exception { Delete delete = new Delete(TEST_ROW).deleteFamily(TEST_FAMILY); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.delete(delete); } finally { t.close(); + connection.close(); } return null; } @@ -362,12 +385,14 @@ public class TestCellACLs extends SecureTestUtil { AccessTestAction deleteQ1 = new AccessTestAction() { @Override public Object run() throws Exception { - Delete delete = new Delete(TEST_ROW).deleteColumn(TEST_FAMILY, TEST_Q1); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Delete delete = new Delete(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { t.delete(delete); } finally { t.close(); + connection.close(); } return null; } @@ -394,7 +419,7 @@ public class TestCellACLs extends SecureTestUtil { verifyDenied(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO); @@ -410,7 +435,7 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO); @@ -426,7 +451,7 @@ public class TestCellACLs extends SecureTestUtil { verifyDenied(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { Put p; p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ONE); @@ -442,7 +467,7 @@ public class TestCellACLs extends SecureTestUtil { verifyAllowed(new AccessTestAction() { @Override public Object run() throws Exception { - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Table t = TEST_UTIL.getConnection().getTable(TEST_TABLE.getTableName()); try { return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java index 327024705e1..887b56d19c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java @@ -157,7 +157,7 @@ public class TestNamespaceCommands extends SecureTestUtil { @Test public void testAclTableEntries() throws Exception { String userTestNamespace = "userTestNsp"; - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Table acl = UTIL.getConnection().getTable(AccessControlLists.ACL_TABLE_NAME); try { ListMultimap perms = AccessControlLists.getNamespacePermissions(conf, TEST_NAMESPACE); @@ -360,7 +360,8 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction grantAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); @@ -369,6 +370,7 @@ public class TestNamespaceCommands extends SecureTestUtil { ProtobufUtil.grant(protocol, testUser, TEST_NAMESPACE, Action.WRITE); } finally { acl.close(); + connection.close(); } return null; } @@ -376,7 +378,8 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction revokeAction = new AccessTestAction() { public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); @@ -385,6 +388,7 @@ public class TestNamespaceCommands extends SecureTestUtil { ProtobufUtil.revoke(protocol, testUser, TEST_NAMESPACE, Action.WRITE); } finally { acl.close(); + connection.close(); } return null; } @@ -393,7 +397,8 @@ public class TestNamespaceCommands extends SecureTestUtil { AccessTestAction getPermissionsAction = new AccessTestAction() { @Override public Object run() throws Exception { - Table acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); + Connection connection = ConnectionFactory.createConnection(conf); + Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME); try { BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW); AccessControlService.BlockingInterface protocol = @@ -401,6 +406,7 @@ public class TestNamespaceCommands extends SecureTestUtil { ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(TEST_NAMESPACE)); } finally { acl.close(); + connection.close(); } return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java index 7b53a37284a..e1dfdbf7629 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestScanEarlyTermination.java @@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableNotFoundException; 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.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; @@ -155,7 +157,8 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Put put = new Put(TEST_ROW).add(TEST_FAMILY1, TEST_Q1, ZERO); t.put(put); @@ -169,6 +172,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { t.put(put); } finally { t.close(); + connection.close(); } return null; } @@ -180,7 +184,8 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Scan scan = new Scan().addFamily(TEST_FAMILY1); Result result = t.getScanner(scan).next(); @@ -192,6 +197,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }, USER_OTHER); @@ -204,7 +210,8 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Scan scan = new Scan(); Result result = t.getScanner(scan).next(); @@ -216,6 +223,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }, USER_OTHER); @@ -226,7 +234,8 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Scan scan = new Scan().addFamily(TEST_FAMILY2); Result result = t.getScanner(scan).next(); @@ -236,6 +245,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }, USER_OTHER); @@ -252,7 +262,8 @@ public class TestScanEarlyTermination extends SecureTestUtil { public Object run() throws Exception { // force a new RS connection conf.set("testkey", UUID.randomUUID().toString()); - Table t = new HTable(conf, TEST_TABLE.getTableName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(TEST_TABLE.getTableName()); try { Scan scan = new Scan(); Result result = t.getScanner(scan).next(); @@ -265,6 +276,7 @@ public class TestScanEarlyTermination extends SecureTestUtil { return null; } finally { t.close(); + connection.close(); } } }, USER_OTHER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java index b795127fd82..34561585e6a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java @@ -288,7 +288,7 @@ public class TestTablePermissions { ListMultimap preperms = AccessControlLists.getTablePermissions(conf, TEST_TABLE); - Table table = new HTable(conf, TEST_TABLE); + Table table = UTIL.getConnection().getTable(TEST_TABLE); table.put(new Put(Bytes.toBytes("row1")) .add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v1"))); table.put(new Put(Bytes.toBytes("row2")) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java index cc615ed45f6..f8bd1177629 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/ExpAsStringVisibilityLabelServiceImpl.java @@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.HConstants.OperationStatusCode; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.classification.InterfaceAudience; +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.HTable; @@ -146,14 +148,19 @@ public class ExpAsStringVisibilityLabelServiceImpl implements VisibilityLabelSer List cells = null; if (labelsRegion == null) { Table table = null; + Connection connection = null; try { - table = new HTable(conf, VisibilityConstants.LABELS_TABLE_NAME); + connection = ConnectionFactory.createConnection(conf); + table = connection.getTable(VisibilityConstants.LABELS_TABLE_NAME); Result result = table.get(get); cells = result.listCells(); } finally { if (table != null) { table.close(); } + if (connection != null){ + connection.close(); + } } } else { cells = this.labelsRegion.get(get, false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java index 2897048ac22..061db74c735 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestDefaultScanLabelGeneratorStack.java @@ -106,6 +106,7 @@ public class TestDefaultScanLabelGeneratorStack { SUPERUSER.runAs(new PrivilegedExceptionAction() { public Void run() throws Exception { + Connection connection = ConnectionFactory.createConnection(conf); Table table = TEST_UTIL.createTable(tableName, CF); try { Put put = new Put(ROW_1); @@ -122,6 +123,7 @@ public class TestDefaultScanLabelGeneratorStack { return null; } finally { table.close(); + connection.close(); } } }); @@ -168,7 +170,8 @@ public class TestDefaultScanLabelGeneratorStack { TESTUSER.runAs(new PrivilegedExceptionAction() { public Void run() throws Exception { - Table table = new HTable(conf, tableName); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName); try { // Test scan with no auth attribute Scan s = new Scan(); @@ -238,6 +241,7 @@ public class TestDefaultScanLabelGeneratorStack { return null; } finally { table.close(); + connection.close(); } } }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java index a06f03d2f9f..df165bd1071 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestEnforcingScanLabelGenerator.java @@ -143,7 +143,8 @@ public class TestEnforcingScanLabelGenerator { TESTUSER.runAs(new PrivilegedExceptionAction() { public Void run() throws Exception { - Table table = new HTable(conf, tableName); + Connection connection = ConnectionFactory.createConnection(conf); + Table table = connection.getTable(tableName); try { // Test that we enforce the defined set Get get = new Get(ROW_1); @@ -161,6 +162,7 @@ public class TestEnforcingScanLabelGenerator { return null; } finally { table.close(); + connection.close(); } } }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index 33583de77a4..01e22daab86 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -38,10 +38,13 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -137,7 +140,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); TEST_UTIL1.startMiniCluster(1); HBaseAdmin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); - HTableDescriptor table = new HTableDescriptor(TableName.valueOf(TABLE_NAME)); + HTableDescriptor table = new HTableDescriptor(TABLE_NAME); HColumnDescriptor desc = new HColumnDescriptor(fam); desc.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); table.addFamily(desc); @@ -171,11 +174,13 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit final boolean nullExpected, final String... auths) throws IOException, InterruptedException { PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() { - HTable table2 = null; + Table table2 = null; public Void run() throws Exception { + Connection connection = null; try { - table2 = new HTable(conf1, TABLE_NAME_BYTES); + connection = ConnectionFactory.createConnection(conf1); + table2 = connection.getTable(TABLE_NAME); CellScanner cellScanner; Cell current; Get get = new Get(row); @@ -205,6 +210,9 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit if (table2 != null) { table2.close(); } + if(connection != null){ + connection.close(); + } } } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 455d63203d5..dcd5fbb459d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -42,6 +42,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Increment; @@ -378,7 +380,7 @@ public abstract class TestVisibilityLabels { t.join(); Table table = null; try { - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Scan s = new Scan(); s.setAuthorizations(new Authorizations(SECRET)); ResultScanner scanner = table.getScanner(s); @@ -472,7 +474,7 @@ public abstract class TestVisibilityLabels { SUPERUSER.runAs(action); Table ht = null; try { - ht = new HTable(conf, LABELS_TABLE_NAME); + ht = TEST_UTIL.getConnection().getTable(LABELS_TABLE_NAME); Scan scan = new Scan(); scan.setAuthorizations(new Authorizations(VisibilityUtils.SYSTEM_LABEL)); ResultScanner scanner = ht.getScanner(scan); @@ -581,9 +583,11 @@ public abstract class TestVisibilityLabels { "org.apache.hadoop.hbase.security.visibility.InvalidLabelException: " + "Label 'public' is not set for the user testUser")); assertTrue(resultList.get(2).getException().getValue().isEmpty()); + Connection connection = null; Table ht = null; try { - ht = new HTable(conf, LABELS_TABLE_NAME); + connection = ConnectionFactory.createConnection(conf); + ht = connection.getTable(LABELS_TABLE_NAME); ResultScanner scanner = ht.getScanner(new Scan()); Result result = null; List results = new ArrayList(); @@ -597,6 +601,9 @@ public abstract class TestVisibilityLabels { if (ht != null) { ht.close(); } + if (connection != null){ + connection.close(); + } } GetAuthsResponse authsResponse = null; @@ -774,7 +781,7 @@ public abstract class TestVisibilityLabels { TEST_UTIL.getHBaseAdmin().createTable(desc); Table table = null; try { - table = new HTable(TEST_UTIL.getConfiguration(), tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put put = new Put(r1); put.add(fam, qual, 3l, v1); put.add(fam, qual2, 3l, v1); @@ -862,7 +869,7 @@ public abstract class TestVisibilityLabels { HColumnDescriptor col = new HColumnDescriptor(fam); desc.addFamily(col); TEST_UTIL.getHBaseAdmin().createTable(desc); - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); try { Put p1 = new Put(row1); p1.add(fam, qual, value); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index 899e63d9ba6..9ea64d107b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HBaseAdmin; @@ -85,8 +87,7 @@ public class TestVisibilityLabelsReplication { protected static final String TEMP = "temp"; protected static Configuration conf; protected static Configuration conf1; - protected static String TABLE_NAME = "TABLE_NAME"; - protected static byte[] TABLE_NAME_BYTES = Bytes.toBytes(TABLE_NAME); + protected static TableName TABLE_NAME = TableName.valueOf("TABLE_NAME"); protected static ReplicationAdmin replicationAdmin; public static final String TOPSECRET = "topsecret"; public static final String PUBLIC = "public"; @@ -187,7 +188,7 @@ public class TestVisibilityLabelsReplication { TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000); TEST_UTIL1.startMiniCluster(1); HBaseAdmin hBaseAdmin = TEST_UTIL.getHBaseAdmin(); - HTableDescriptor table = new HTableDescriptor(TableName.valueOf(TABLE_NAME)); + HTableDescriptor table = new HTableDescriptor(TABLE_NAME); HColumnDescriptor desc = new HColumnDescriptor(fam); desc.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); table.addFamily(desc); @@ -218,8 +219,7 @@ public class TestVisibilityLabelsReplication { @Test public void testVisibilityReplication() throws Exception { - TableName tableName = TableName.valueOf(TABLE_NAME); - Table table = writeData(tableName, "(" + SECRET + "&" + PUBLIC + ")" + "|(" + CONFIDENTIAL + Table table = writeData(TABLE_NAME, "(" + SECRET + "&" + PUBLIC + ")" + "|(" + CONFIDENTIAL + ")&(" + TOPSECRET + ")", "(" + PRIVATE + "|" + CONFIDENTIAL + ")&(" + PUBLIC + "|" + TOPSECRET + ")", "(" + SECRET + "|" + CONFIDENTIAL + ")" + "&" + "!" + TOPSECRET, CellVisibility.quote(UNICODE_VIS_TAG) + "&" + SECRET); @@ -252,9 +252,9 @@ public class TestVisibilityLabelsReplication { current = cellScanner.current(); assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(), row4, 0, row4.length)); - HTable table2 = null; + Table table2 = null; try { - table2 = new HTable(TEST_UTIL1.getConfiguration(), TABLE_NAME_BYTES); + table2 = TEST_UTIL1.getConnection().getTable(TABLE_NAME); s = new Scan(); // Ensure both rows are replicated scanner = table2.getScanner(s); @@ -314,11 +314,13 @@ public class TestVisibilityLabelsReplication { final boolean nullExpected, final String... auths) throws IOException, InterruptedException { PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() { - HTable table2 = null; + Table table2 = null; + Connection connection = null; public Void run() throws Exception { try { - table2 = new HTable(conf1, TABLE_NAME_BYTES); + connection = ConnectionFactory.createConnection(conf1); + table2 = connection.getTable(TABLE_NAME); CellScanner cellScanner; Cell current; Get get = new Get(row); @@ -356,6 +358,9 @@ public class TestVisibilityLabelsReplication { if (table2 != null) { table2.close(); } + if(connection != null) { + connection.close(); + } } } }; @@ -394,9 +399,9 @@ public class TestVisibilityLabelsReplication { } static Table writeData(TableName tableName, String... labelExps) throws Exception { - HTable table = null; + Table table = null; try { - table = new HTable(conf, TABLE_NAME_BYTES); + table = TEST_UTIL.getConnection().getTable(TABLE_NAME); int i = 1; List puts = new ArrayList(); for (String labelExp : labelExps) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java index d4f5d67c2e4..7e7d8a388d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithACL.java @@ -31,6 +31,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; @@ -124,7 +126,8 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Scan s = new Scan(); s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - Table t = new HTable(conf, table.getName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(table.getName()); try { ResultScanner scanner = t.getScanner(s); Result result = scanner.next(); @@ -134,6 +137,7 @@ public class TestVisibilityLabelsWithACL { assertNull(result); } finally { t.close(); + connection.close(); } return null; } @@ -153,7 +157,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Scan s = new Scan(); s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - Table t = new HTable(conf, table.getName()); + Table t = TEST_UTIL.getConnection().getTable(table.getName()); try { ResultScanner scanner = t.getScanner(s); Result[] result = scanner.next(5); @@ -179,7 +183,7 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Get g = new Get(row1); g.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - Table t = new HTable(conf, table.getName()); + Table t = TEST_UTIL.getConnection().getTable(table.getName()); try { Result result = t.get(g); assertTrue(!result.isEmpty()); @@ -208,12 +212,14 @@ public class TestVisibilityLabelsWithACL { public Void run() throws Exception { Get g = new Get(row1); g.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL)); - Table t = new HTable(conf, table.getName()); + Connection connection = ConnectionFactory.createConnection(conf); + Table t = connection.getTable(table.getName()); try { Result result = t.get(g); assertTrue(result.isEmpty()); } finally { t.close(); + connection.close(); } return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java index dcfed5ffce5..52f86c3911c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java @@ -150,7 +150,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili // Scan the visibility label Scan s = new Scan(); s.setAuthorizations(new Authorizations(VisibilityUtils.SYSTEM_LABEL)); - Table ht = new HTable(conf, LABELS_TABLE_NAME); + Table ht = TEST_UTIL.getConnection().getTable(LABELS_TABLE_NAME); int i = 0; try { ResultScanner scanner = ht.getScanner(s); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java index 6029c8b0c5c..b2d0ae5f740 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java @@ -17,17 +17,6 @@ */ package org.apache.hadoop.hbase.security.visibility; -import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.io.InterruptedIOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; @@ -38,7 +27,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -58,6 +46,17 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + /** * Tests visibility labels with deletes */ @@ -119,7 +118,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET)); d.addColumns(fam, qual); @@ -161,7 +160,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row2); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.addFamily(fam); @@ -203,7 +202,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.deleteFamilyVersion(fam, 123l); @@ -245,7 +244,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL)); d.addColumn(fam, qual, 123l); @@ -285,7 +284,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -340,7 +339,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d1 = new Delete(row1); d1.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d1.addColumns(fam, qual); @@ -389,7 +388,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addColumns(fam, qual); @@ -441,7 +440,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d1 = new Delete(row1); d1.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d1.addFamily(fam); @@ -487,7 +486,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -500,7 +499,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addFamily(fam); @@ -525,7 +524,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.addFamily(fam); @@ -561,7 +560,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -574,7 +573,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addColumns(fam, qual); @@ -599,7 +598,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.addColumns(fam, qual); @@ -635,7 +634,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual, 123l, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -647,7 +646,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.addColumns(fam, qual, 126l); @@ -656,7 +655,7 @@ public class TestVisibilityLabelsWithDeletes { throw new IOException(t); } - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addColumn(fam, qual, 123l); @@ -687,7 +686,7 @@ public class TestVisibilityLabelsWithDeletes { desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put1 = new Put(Bytes.toBytes("row1")); put1.add(fam, qual, 123l, value); put1.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -708,14 +707,14 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addColumn(fam, qual, 123l); table.delete(d); } - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET)); d.addColumn(fam, qual, 123l); @@ -760,7 +759,7 @@ public class TestVisibilityLabelsWithDeletes { + SECRET + "&" + TOPSECRET + ")")); d3.addFamily(fam); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.delete(createList(d1, d2, d3)); } catch (Throwable t) { throw new IOException(t); @@ -804,7 +803,7 @@ public class TestVisibilityLabelsWithDeletes { Delete d2 = new Delete(row1); d2.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d2.addColumns(fam, qual); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.delete(createList(d1, d2)); } catch (Throwable t) { throw new IOException(t); @@ -894,7 +893,7 @@ public class TestVisibilityLabelsWithDeletes { + "&" + SECRET + ")")); puts.add(put); - Table table = new HTable(conf, tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); table.put(puts); return table; } @@ -936,7 +935,7 @@ public class TestVisibilityLabelsWithDeletes { + TOPSECRET + "&" + SECRET+")")); puts.add(put); - Table table = new HTable(conf, tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); table.put(puts); return table; } @@ -970,7 +969,7 @@ public class TestVisibilityLabelsWithDeletes { put.add(fam, qual, 127l, value); puts.add(put); - Table table = new HTable(conf, tableName); + Table table = TEST_UTIL.getConnection().getTable(tableName); table.put(puts); TEST_UTIL.getHBaseAdmin().flush(tableName); @@ -993,7 +992,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1057,7 +1056,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumn(fam, qual); @@ -1121,7 +1120,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET )); d.addColumn(fam, qual); @@ -1209,7 +1208,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumn(fam, qual); @@ -1280,7 +1279,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addFamily(fam); @@ -1337,7 +1336,7 @@ public class TestVisibilityLabelsWithDeletes { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumns(fam, qual, 125l); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.delete(d); } catch (Throwable t) { throw new IOException(t); @@ -1395,7 +1394,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual1, 125l, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -1416,7 +1415,7 @@ public class TestVisibilityLabelsWithDeletes { d2.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d2.addColumns(fam, qual1, 125l); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.delete(createList(d1, d2)); } catch (Throwable t) { throw new IOException(t); @@ -1442,7 +1441,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual1, 125l, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -1463,7 +1462,7 @@ public class TestVisibilityLabelsWithDeletes { d2.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d2.addColumns(fam, qual1, 126l); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { table.delete(createList(d1, d2)); } catch (Throwable t) { throw new IOException(t); @@ -1491,7 +1490,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addFamily(fam); table.delete(d); @@ -1534,7 +1533,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addFamily(fam); @@ -1602,7 +1601,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")")); @@ -1662,7 +1661,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET+")")); @@ -1720,7 +1719,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1771,7 +1770,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -1834,7 +1833,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.deleteFamilyVersion(fam, 123l); @@ -1885,7 +1884,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -1930,7 +1929,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addFamily(fam); @@ -1982,7 +1981,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2028,7 +2027,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(CONFIDENTIAL)); d.addFamily(fam); @@ -2068,7 +2067,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumn(fam, qual, 125l); @@ -2118,7 +2117,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2173,7 +2172,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")" + "|(" + TOPSECRET + "&" + SECRET + ")")); @@ -2224,7 +2223,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumn(fam, qual, 127l); @@ -2286,7 +2285,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2342,7 +2341,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2406,7 +2405,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET)); d.addColumn(fam, qual, 125l); @@ -2456,7 +2455,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2524,7 +2523,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2575,7 +2574,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET+")")); @@ -2627,7 +2626,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addColumn(fam, qual, 125l); table.delete(d); @@ -2650,7 +2649,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addColumns(fam, qual, 125l); table.delete(d); @@ -2674,7 +2673,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addFamily(fam, 125l); table.delete(d); @@ -2698,7 +2697,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addFamily(fam); table.delete(d); @@ -2722,7 +2721,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addColumns(fam, qual); table.delete(d); @@ -2746,7 +2745,7 @@ public class TestVisibilityLabelsWithDeletes { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.deleteFamilyVersion(fam, 126l); table.delete(d); @@ -2813,7 +2812,7 @@ public class TestVisibilityLabelsWithDeletes { HTableDescriptor desc = new HTableDescriptor(tableName); desc.addFamily(colDesc); hBaseAdmin.createTable(desc); - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Put put = new Put(Bytes.toBytes("row1")); put.add(fam, qual, 123l, value); put.setCellVisibility(new CellVisibility(CONFIDENTIAL)); @@ -2826,7 +2825,7 @@ public class TestVisibilityLabelsWithDeletes { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - try (Table table = new HTable(conf, tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { Delete d = new Delete(row1); d.addColumn(fam, qual, 124l); d.setCellVisibility(new CellVisibility(PRIVATE )); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java index 828c89b2ccd..457d2eb3e20 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityWithCheckAuths.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -135,7 +137,7 @@ public class TestVisibilityWithCheckAuths { public Void run() throws Exception { Table table = null; try { - table = new HTable(conf, tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Put p = new Put(row1); p.setCellVisibility(new CellVisibility(PUBLIC + "&" + TOPSECRET)); p.add(fam, qual, 125l, value); @@ -179,15 +181,17 @@ public class TestVisibilityWithCheckAuths { PrivilegedExceptionAction actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { + Connection connection = ConnectionFactory.createConnection(conf); Table table = null; try { - table = new HTable(conf, tableName); + table = connection.getTable(tableName); Put put = new Put(row1); put.add(fam, qual, HConstants.LATEST_TIMESTAMP, val); put.setCellVisibility(new CellVisibility(TOPSECRET)); table.put(put); } finally { table.close(); + connection.close(); } return null; } @@ -196,14 +200,16 @@ public class TestVisibilityWithCheckAuths { actiona = new PrivilegedExceptionAction() { @Override public Void run() throws Exception { + Connection connection = ConnectionFactory.createConnection(conf); Table table = null; try { - table = new HTable(conf, tableName); + table = TEST_UTIL.getConnection().getTable(tableName); Append append = new Append(row1); append.add(fam, qual, Bytes.toBytes("b")); table.append(append); } finally { table.close(); + connection.close(); } return null; } @@ -213,8 +219,10 @@ public class TestVisibilityWithCheckAuths { @Override public Void run() throws Exception { Table table = null; + Connection connection = null; try { - table = new HTable(conf, tableName); + connection = ConnectionFactory.createConnection(conf); + table = connection.getTable(tableName); Append append = new Append(row1); append.add(fam, qual, Bytes.toBytes("c")); append.setCellVisibility(new CellVisibility(PUBLIC)); @@ -223,7 +231,12 @@ public class TestVisibilityWithCheckAuths { } catch (Throwable t) { assertTrue(t.getMessage().contains("AccessDeniedException")); } finally { - table.close(); + if (table != null) { + table.close(); + } + if (connection != null) { + connection.close(); + } } return null; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index 44f411f1e60..74f358d255a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -673,10 +673,10 @@ public class SnapshotTestingUtils { public static void loadData(final HBaseTestingUtility util, final TableName tableName, int rows, byte[]... families) throws IOException, InterruptedException { - loadData(util, new HTable(util.getConfiguration(), tableName), rows, families); + loadData(util, util.getConnection().getTable(tableName), rows, families); } - public static void loadData(final HBaseTestingUtility util, final HTable table, int rows, + public static void loadData(final HBaseTestingUtility util, final Table table, int rows, byte[]... families) throws IOException, InterruptedException { table.setAutoFlushTo(false); @@ -731,7 +731,7 @@ public class SnapshotTestingUtils { public static void verifyRowCount(final HBaseTestingUtility util, final TableName tableName, long expectedRows) throws IOException { - Table table = new HTable(util.getConfiguration(), tableName); + Table table = util.getConnection().getTable(tableName); try { assertEquals(expectedRows, util.countRows(table)); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java index 192009b8c1b..2774bee6978 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java @@ -117,7 +117,6 @@ public class TestExportSnapshot { admin.snapshot(emptySnapshotName, tableName); // Add some rows - Table table = new HTable(TEST_UTIL.getConfiguration(), tableName); SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY); tableNumFiles = admin.getTableRegions(tableName).size(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java index b96fab6e5d8..38938714bfc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.ScannerCallable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.ipc.AbstractRpcClient; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.HMaster; @@ -144,7 +145,7 @@ public class TestFlushSnapshotFromClient { SnapshotTestingUtils.assertNoSnapshots(admin); // put some stuff in the table - HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME); + Table table = UTIL.getConnection().getTable(TABLE_NAME); SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM); LOG.debug("FS state before snapshot:"); @@ -183,7 +184,7 @@ public class TestFlushSnapshotFromClient { SnapshotTestingUtils.assertNoSnapshots(admin); // put some stuff in the table - HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME); + Table table = UTIL.getConnection().getTable(TABLE_NAME); UTIL.loadTable(table, TEST_FAM); LOG.debug("FS state before snapshot:"); @@ -227,7 +228,7 @@ public class TestFlushSnapshotFromClient { SnapshotTestingUtils.assertNoSnapshots(admin); // put some stuff in the table - HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME); + Table table = UTIL.getConnection().getTable(TABLE_NAME); SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM); LOG.debug("FS state before snapshot:"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java index e6bc2055aa1..4b36c114125 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; @@ -102,7 +103,7 @@ public class TestRestoreFlushSnapshotFromClient { // create Table and disable it SnapshotTestingUtils.createTable(UTIL, tableName, FAMILY); - HTable table = new HTable(UTIL.getConfiguration(), tableName); + Table table = UTIL.getConnection().getTable(tableName); SnapshotTestingUtils.loadData(UTIL, table, 500, FAMILY); snapshot0Rows = UTIL.countRows(table); LOG.info("=== before snapshot with 500 rows"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java index 8635c11c0c6..4af63c1cbe1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedReaderWithACL.java @@ -101,7 +101,7 @@ public class MultiThreadedReaderWithACL extends MultiThreadedReader { int specialPermCellInsertionFactor = Integer.parseInt(dataGenerator.getArgs()[2]); int mod = ((int) keyToRead % userNames.length); if (userVsTable.get(userNames[mod]) == null) { - localTable = new HTable(conf, tableName); + localTable = connection.getTable(tableName); userVsTable.put(userNames[mod], localTable); result = localTable.get(get); } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java index 30d425c5a4e..7529727d17e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedUpdaterWithACL.java @@ -118,7 +118,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { try { int mod = ((int) rowKeyBase % userNames.length); if (userVsTable.get(userNames[mod]) == null) { - localTable = new HTable(conf, tableName); + localTable = connection.getTable(tableName); userVsTable.put(userNames[mod], localTable); res = localTable.get(get); } else { @@ -227,7 +227,7 @@ public class MultiThreadedUpdaterWithACL extends MultiThreadedUpdater { public Object run() throws Exception { try { if (table == null) { - table = new HTable(conf, tableName); + table = connection.getTable(tableName); } if (m instanceof Increment) { table.increment((Increment) m); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java index 14765679fcd..f247670f492 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MultiThreadedWriterWithACL.java @@ -127,7 +127,7 @@ public class MultiThreadedWriterWithACL extends MultiThreadedWriter { public Object run() throws Exception { try { if (table == null) { - table = new HTable(conf, tableName); + table = connection.getTable(tableName); } table.put(put); } catch (IOException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index be7ec79446d..9f98624bd6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -173,7 +173,7 @@ public class ProcessBasedLocalHBaseCluster { int attemptsLeft = 10; while (attemptsLeft-- > 0) { try { - new HTable(conf, TableName.META_TABLE_NAME); + testUtil.getConnection().getTable(TableName.META_TABLE_NAME); } catch (Exception e) { LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft, e); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index 81b219d6198..b0a17a91b99 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -24,6 +24,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -109,7 +111,9 @@ public class RestartMetaTest extends AbstractHBaseTool { " seconds....\n\n"); Threads.sleep(5 * SLEEP_SEC_AFTER_DATA_LOAD); - int metaRSPort = HBaseTestingUtility.getMetaRSPort(conf); + Connection connection = ConnectionFactory.createConnection(conf); + + int metaRSPort = HBaseTestingUtility.getMetaRSPort(connection); LOG.debug("Killing hbase:meta region server running on port " + metaRSPort); hbaseCluster.killRegionServer(metaRSPort); @@ -121,7 +125,7 @@ public class RestartMetaTest extends AbstractHBaseTool { LOG.debug("Trying to scan meta"); - Table metaTable = new HTable(conf, TableName.META_TABLE_NAME); + Table metaTable = connection.getTable(TableName.META_TABLE_NAME); ResultScanner scanner = metaTable.getScanner(new Scan()); Result result; while ((result = scanner.next()) != null) { @@ -131,6 +135,8 @@ public class RestartMetaTest extends AbstractHBaseTool { + Bytes.toStringBinary(result.getFamilyMap(HConstants.CATALOG_FAMILY) .get(HConstants.SERVER_QUALIFIER))); } + metaTable.close(); + connection.close(); return 0; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index 034d6bcdfcc..9a7db90104a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -166,7 +166,7 @@ public class TestCoprocessorScanPolicy { .setTimeToLive(1); desc.addFamily(hcd); TEST_UTIL.getHBaseAdmin().createTable(desc); - Table t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + Table t = TEST_UTIL.getConnection().getTable(tableName); long now = EnvironmentEdgeManager.currentTime(); ManualEnvironmentEdge me = new ManualEnvironmentEdge(); me.setValue(now); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index 27de51ddf33..69ffa5593ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -103,7 +103,7 @@ public class TestHBaseFsckEncryption { @Test public void testFsckWithEncryption() throws Exception { // Populate the table with some data - Table table = new HTable(conf, htd.getTableName()); + Table table = TEST_UTIL.getConnection().getTable(htd.getTableName()); try { byte[] values = { 'A', 'B', 'C', 'D' }; for (int i = 0; i < values.length; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java index 89dfbc10f25..69e5d7017d7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMergeTable.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; 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.HBaseAdmin; import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.Put; @@ -114,14 +115,15 @@ public class TestMergeTable { LOG.info("Starting mini hbase cluster"); UTIL.startMiniHBaseCluster(1, 1); Configuration c = new Configuration(UTIL.getConfiguration()); - Connection connection = HConnectionManager.getConnection(c); + Connection connection = UTIL.getConnection(); List originalTableRegions = MetaTableAccessor.getTableRegions(connection, desc.getTableName()); LOG.info("originalTableRegions size=" + originalTableRegions.size() + "; " + originalTableRegions); - Admin admin = new HBaseAdmin(c); + Admin admin = connection.getAdmin(); admin.disableTable(desc.getTableName()); + admin.close(); HMerge.merge(c, FileSystem.get(c), desc.getTableName()); List postMergeTableRegions = MetaTableAccessor.getTableRegions(connection, desc.getTableName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java index d10ce1e7d94..68d4b37dd64 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java @@ -152,7 +152,7 @@ public class TestMiniClusterLoadSequential { LOG.info("Starting load test: dataBlockEncoding=" + dataBlockEncoding + ", isMultiPut=" + isMultiPut); numKeys = numKeys(); - Admin admin = new HBaseAdmin(conf); + Admin admin = TEST_UTIL.getHBaseAdmin(); while (admin.getClusterStatus().getServers().size() < NUM_RS) { LOG.info("Sleeping until " + NUM_RS + " RSs are online"); Threads.sleepWithoutInterrupt(1000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java index e8d22b82003..513d53864c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java @@ -62,7 +62,7 @@ public class TestProcessBasedCluster { HTestConst.DEFAULT_CF_STR_SET, HColumnDescriptor.DEFAULT_VERSIONS, COLS_PER_ROW, FLUSHES, NUM_REGIONS, ROWS_PER_FLUSH); - Table table = new HTable(TEST_UTIL.getConfiguration(), HTestConst.DEFAULT_TABLE); + Table table = TEST_UTIL.getConnection().getTable(HTestConst.DEFAULT_TABLE); ResultScanner scanner = table.getScanner(HTestConst.DEFAULT_CF_BYTES); Result result; int rows = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java index 432c4b36307..63154a80f21 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java @@ -310,9 +310,8 @@ public class TestRegionSplitter { private void verifyBounds(List expectedBounds, TableName tableName) throws Exception { // Get region boundaries from the cluster and verify their endpoints - final Configuration conf = UTIL.getConfiguration(); final int numRegions = expectedBounds.size()-1; - final HTable hTable = new HTable(conf, tableName); + final HTable hTable = (HTable) UTIL.getConnection().getTable(tableName); final Map regionInfoMap = hTable.getRegionLocations(); assertEquals(numRegions, regionInfoMap.size()); for (Map.Entry entry: regionInfoMap.entrySet()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java index 6c33a9bd694..349bf567e60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java @@ -215,7 +215,7 @@ public class OfflineMetaRebuildTestCore { protected HRegionInfo createRegion(Configuration conf, final Table htbl, byte[] startKey, byte[] endKey) throws IOException { - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); HTableDescriptor htd = htbl.getTableDescriptor(); HRegionInfo hri = new HRegionInfo(htbl.getName(), startKey, endKey); @@ -240,7 +240,7 @@ public class OfflineMetaRebuildTestCore { // Mess it up by blowing up meta. Admin admin = TEST_UTIL.getHBaseAdmin(); Scan s = new Scan(); - Table meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); ResultScanner scanner = meta.getScanner(s); List dels = new ArrayList(); for (Result r : scanner) { @@ -266,7 +266,7 @@ public class OfflineMetaRebuildTestCore { */ protected int tableRowCount(Configuration conf, TableName table) throws IOException { - Table t = new HTable(conf, table); + Table t = TEST_UTIL.getConnection().getTable(table); Scan st = new Scan(); ResultScanner rst = t.getScanner(st); @@ -286,9 +286,9 @@ public class OfflineMetaRebuildTestCore { */ protected int scanMeta() throws IOException { int count = 0; - HTable meta = new HTable(conf, TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); ResultScanner scanner = meta.getScanner(new Scan()); - LOG.info("Table: " + Bytes.toString(meta.getTableName())); + LOG.info("Table: " + meta.getName()); for (Result res : scanner) { LOG.info(Bytes.toString(res.getRow())); count++; diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java index 400f10f0013..e9c9e1fec8f 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/HTablePool.java @@ -29,7 +29,20 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HTableFactory; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.HTableInterfaceFactory; +import org.apache.hadoop.hbase.client.Increment; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; @@ -220,8 +233,7 @@ public class HTablePool implements Closeable { * @param tableName * table name * @return a reference to the specified table - * @throws RuntimeException - * if there is a problem instantiating the HTable + * @throws RuntimeException if there is a problem instantiating the HTable */ public HTableInterface getTable(byte[] tableName) { return getTable(Bytes.toString(tableName)); @@ -645,7 +657,7 @@ public class HTablePool implements Closeable { private void checkState() { if (!isOpen()) { - throw new IllegalStateException("Table=" + new String(table.getTableName()) + throw new IllegalStateException("Table=" + table.getName() + " already closed"); } } diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java index 2826b05e2bc..101a7cf4719 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestHTablePool.java @@ -254,7 +254,7 @@ public class TestHTablePool { public void testCloseTablePool() throws IOException { HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 4, getPoolType()); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); if (admin.tableExists(TABLENAME)) { admin.disableTable(TABLENAME); @@ -330,7 +330,7 @@ public class TestHTablePool { public void testCloseTablePool() throws IOException { HTablePool pool = new HTablePool(TEST_UTIL.getConfiguration(), 4, getPoolType()); - HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration()); + HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); if (admin.tableExists(TABLENAME)) { admin.disableTable(TABLENAME); 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 d3de6dd1d53..f4df2713dcb 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 @@ -135,7 +135,7 @@ public class TestThriftHBaseServiceHandler { @BeforeClass public static void beforeClass() throws Exception { UTIL.startMiniCluster(); - Admin admin = new HBaseAdmin(UTIL.getConfiguration()); + Admin admin = UTIL.getHBaseAdmin(); HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableAname)); for (HColumnDescriptor family : families) { tableDescriptor.addFamily(family); diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java index 80c54df9eec..42d1b08dd57 100644 --- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java +++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandlerWithLabels.java @@ -138,7 +138,7 @@ public static void beforeClass() throws Exception { // Wait for the labels table to become available UTIL.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000); createLabels(); - Admin admin = new HBaseAdmin(UTIL.getConfiguration()); + Admin admin = UTIL.getHBaseAdmin(); HTableDescriptor tableDescriptor = new HTableDescriptor( TableName.valueOf(tableAname)); for (HColumnDescriptor family : families) {